Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Started by li jieabout 2 years ago54 messages
#1li jie
ggysxcq@gmail.com

Hi,

During logical decoding, if there is a large write transaction, some
spill files will be written to disk,
depending on the setting of max_changes_in_memory.

This behavior can effectively avoid OOM, but if the transaction
generates a lot of change before commit,
a large number of files may fill the disk. For example, you can update
a TB-level table.
Of course, this is also inevitable.

But I found an inelegant phenomenon. If the updated large table is not
published, its changes will also
be written with a large number of spill files. Look at an example below:

publisher:
```
create table tbl_pub(id int, val1 text, val2 text,val3 text);
create table tbl_t1(id int, val1 text, val2 text,val3 text);
CREATE PUBLICATION mypub FOR TABLE public.tbl_pub;
```

subscriber:
```
create table tbl_pub(id int, val1 text, val2 text,val3 text);
create table tbl_t1(id int, val1 text, val2 text,val3 text);
CREATE SUBSCRIPTION mysub CONNECTION 'host=127.0.0.1 port=5432
user=postgres dbname=postgres' PUBLICATION mypub;
```

publisher:
```
begin;
insert into tbl_t1 select i,repeat('xyzzy', i),repeat('abcba',
i),repeat('dfds', i) from generate_series(0,999999) i;
```

Later you will see a large number of spill files in the
"/$PGDATA/pg_replslot/mysub/" directory.
```
$ll -sh
total 4.5G
4.0K -rw------- 1 postgres postgres 200 Nov 30 09:24 state
17M -rw------- 1 postgres postgres 17M Nov 30 08:22 xid-750-lsn-0-10000000.spill
12M -rw------- 1 postgres postgres 12M Nov 30 08:20 xid-750-lsn-0-1000000.spill
17M -rw------- 1 postgres postgres 17M Nov 30 08:23 xid-750-lsn-0-11000000.spill
......
```

We can see that table tbl_t1 is not published in mypub. It is also not
sent downstream because it is subscribed.
After the transaction is reorganized, the pgoutput decoding plug-in
filters out these change of unpublished relation
when sending logical changes. see function pgoutput_change.

Above all, if after constructing a change and before queuing a change
into a transaction, we filter out unpublished
relation-related changes, will it make logical decoding less laborious
and avoid disk growth as much as possible?

This is just an immature idea. I haven't started to implement it yet.
Maybe it was designed this way because there
are key factors that I didn't consider. So I want to hear everyone's
opinions, especially the designers of logic decoding.

#2li jie
ggysxcq@gmail.com
In reply to: li jie (#1)
1 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

This is just an immature idea. I haven't started to implement it yet.
Maybe it was designed this way because there
are key factors that I didn't consider. So I want to hear everyone's
opinions, especially the designers of logic decoding.

Attached is the patch I used to implement this optimization.
The main designs are as follows:
1. Added a callback LogicalDecodeFilterByRelCB for the output plugin.

2. Added this callback function pgoutput_table_filter for the pgoutput plugin.
Its main implementation is based on the table filter in the
pgoutput_change function.

3. After constructing a change and before Queue a change into a transaction,
use RelidByRelfilenumber to obtain the relation associated with the change,
just like obtaining the relation in the ReorderBufferProcessTXN function.

4. Relation may be a toast, and there is no good way to get its real
table relation
based on toast relation. Here, I get the real table oid through
toast relname, and
then get the real table relation.

5. This filtering takes into account INSERT/UPDATE/INSERT. Other
changes have not
been considered yet and can be expanded in the future.

6. The table filter in pgoutput_change and the get relation in
ReorderBufferProcessTXN
can be deleted. This has not been done yet. This is the next step.

Sincerely look forward to your feedback.
Regards, lijie

Attachments:

v1-Filter-irrelevant-change-before-reassemble-transacti.patchapplication/octet-stream; name=v1-Filter-irrelevant-change-before-reassemble-transacti.patchDownload
From b4ac04d5fd7e87ee199f996dfda96305a6dac967 Mon Sep 17 00:00:00 2001
From: "adger.lj" <adger.lj@alibaba-inc.com>
Date: Fri, 1 Dec 2023 13:58:33 +0800
Subject: [PATCH] Filter irrelevant change before reassemble transactions
 during logical decoding

In order to reduce unnecessary logical decoding, irrelevant relationship
changes can be filtered out before reorganizing transaction fragments.
This can effectively reduce useless changes and prevent disk space from
being filled up with irrelevant data.

By design, Added a callback LogicalDecodeFilterByRelCB for the output plugin.
We implemented a function pgoutput_table_filter for pgoutput. And RelationSyncCache
is reused to determine whether a relationship-related change should be filtered out.

Referring to the implementation of the function pgoutput_change, currently only
insert/update/delete is can filtered, and other types of changes are not considered.
Perhaps more detailed analysis can be done and more filters can be filtered.

The filtering in pgoutput_change and ReorderBufferProcessTXN is no longer needed,
but is retained for now. Here's what to do next.
---
 src/backend/replication/logical/decode.c    | 156 +++++++++++++++++++-
 src/backend/replication/logical/logical.c   |  31 ++++
 src/backend/replication/pgoutput/pgoutput.c |  55 +++++++
 src/include/replication/logical.h           |   2 +
 src/include/replication/output_plugin.h     |   8 +
 src/test/subscription/t/034_table_filter.pl |  91 ++++++++++++
 6 files changed, 339 insertions(+), 4 deletions(-)
 create mode 100644 src/test/subscription/t/034_table_filter.pl

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 1237118e84..ae28d2145b 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -35,6 +35,7 @@
 #include "access/xlogrecord.h"
 #include "access/xlogutils.h"
 #include "catalog/pg_control.h"
+#include "common/string.h"
 #include "replication/decode.h"
 #include "replication/logical.h"
 #include "replication/message.h"
@@ -42,6 +43,7 @@
 #include "replication/reorderbuffer.h"
 #include "replication/snapbuild.h"
 #include "storage/standby.h"
+#include "utils/relfilenumbermap.h"
 
 /* individual record(group)'s handlers */
 static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -155,7 +157,7 @@ xlog_decode(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 		case XLOG_PARAMETER_CHANGE:
 			{
 				xl_parameter_change *xlrec =
-					(xl_parameter_change *) XLogRecGetData(buf->record);
+				(xl_parameter_change *) XLogRecGetData(buf->record);
 
 				/*
 				 * If wal_level on the primary is reduced to less than
@@ -581,6 +583,137 @@ FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return filter_by_origin_cb_wrapper(ctx, origin_id);
 }
 
+static bool
+FilterByTable(LogicalDecodingContext *ctx, ReorderBufferChange *change)
+{
+	ReorderBuffer *rb = ctx->reorder;
+	Relation	relation = NULL;
+	Oid			reloid;
+	bool		result = false;
+	bool		using_subtxn;
+
+	if (ctx->callbacks.filter_by_origin_cb == NULL)
+		return false;
+
+	switch (change->action)
+	{
+			/* intentionally fall through */
+		case REORDER_BUFFER_CHANGE_INSERT:
+		case REORDER_BUFFER_CHANGE_UPDATE:
+		case REORDER_BUFFER_CHANGE_DELETE:
+			break;
+		default:
+			return false;
+	}
+
+	/*
+	 * Decoding needs access to syscaches et al., which in turn use
+	 * heavyweight locks and such. Thus we need to have enough state around to
+	 * keep track of those.  The easiest way is to simply use a transaction
+	 * internally.  That also allows us to easily enforce that nothing writes
+	 * to the database by checking for xid assignments.
+	 *
+	 * When we're called via the SQL SRF there's already a transaction
+	 * started, so start an explicit subtransaction there.
+	 */
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by table");
+	else
+		StartTransactionCommand();
+
+	reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
+								  change->data.tp.rlocator.relNumber);
+	if (reloid == InvalidOid)
+		goto filter_done;
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid,
+			 relpathperm(change->data.tp.rlocator,
+						 MAIN_FORKNUM));
+
+	if (!RelationIsLogicallyLogged(relation))
+	{
+		result = true;
+		goto filter_done;
+	}
+
+	/*
+	 * Ignore temporary heaps created during DDL unless the plugin has asked
+	 * for them.
+	 */
+	if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		result = true;
+		goto filter_done;
+	}
+
+	/*
+	 * For now ignore sequence changes entirely. Most of the time they don't
+	 * log changes using records we understand, so it doesn't make sense to
+	 * handle the few cases we do.
+	 */
+	if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		result = true;
+		goto filter_done;
+	}
+
+	if (IsToastRelation(relation))
+	{
+		Oid			real_reloid = InvalidOid;
+
+		/* pg_toast_ len is 9 */
+		char	   *toast_name = RelationGetRelationName(relation);
+		char	   *start_ch = &toast_name[9];
+
+		real_reloid = strtoint(start_ch, NULL, 10);
+
+		if (real_reloid == InvalidOid)
+			goto filter_done;
+
+		RelationClose(relation);
+
+		relation = RelationIdGetRelation(real_reloid);
+
+		if (!RelationIsValid(relation))
+			elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+				 reloid,
+				 relpathperm(change->data.tp.rlocator,
+							 MAIN_FORKNUM));
+	}
+
+	result = filter_by_table_cb_wrapper(ctx, relation, change);
+
+filter_done:
+
+	if (result && RelationIsValid(relation))
+		elog(LOG, "logical filter change by table %s", RelationGetRelationName(relation));
+
+	/* this is just a sanity check against bad output plugin behaviour */
+	if (GetCurrentTransactionIdIfAny() != InvalidTransactionId)
+		elog(ERROR, "output plugin used XID %u",
+			 GetCurrentTransactionId());
+
+	if (RelationIsValid(relation))
+	{
+		RelationClose(relation);
+	}
+
+	AbortCurrentTransaction();
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	if (result)
+		ReorderBufferReturnChange(rb, change, false);
+
+	return result;
+}
+
 /*
  * Handle rmgr LOGICALMSG_ID records for LogicalDecodingProcessRecord().
  */
@@ -940,6 +1073,9 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 	change->data.tp.clear_toast_afterwards = true;
 
+	if (FilterByTable(ctx, change))
+		return;
+
 	ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r), buf->origptr,
 							 change,
 							 xlrec->flags & XLH_INSERT_ON_TOAST_RELATION);
@@ -1009,6 +1145,9 @@ DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 	change->data.tp.clear_toast_afterwards = true;
 
+	if (FilterByTable(ctx, change))
+		return;
+
 	ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r), buf->origptr,
 							 change, false);
 }
@@ -1065,6 +1204,9 @@ DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 	change->data.tp.clear_toast_afterwards = true;
 
+	if (FilterByTable(ctx, change))
+		return;
+
 	ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r), buf->origptr,
 							 change, false);
 }
@@ -1201,11 +1343,14 @@ DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 		else
 			change->data.tp.clear_toast_afterwards = false;
 
-		ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r),
-								 buf->origptr, change, false);
-
 		/* move to the next xl_multi_insert_tuple entry */
 		data += datalen;
+
+		if (FilterByTable(ctx, change))
+			continue;;
+
+		ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r),
+								 buf->origptr, change, false);
 	}
 	Assert(data == tupledata + tuplelen);
 }
@@ -1240,6 +1385,9 @@ DecodeSpecConfirm(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 	change->data.tp.clear_toast_afterwards = true;
 
+	if (FilterByTable(ctx, change))
+		return;
+
 	ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r), buf->origptr,
 							 change, false);
 }
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 8288da5277..f5da5199dc 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -1225,6 +1225,37 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+bool
+filter_by_table_cb_wrapper(LogicalDecodingContext *ctx, Relation relation, ReorderBufferChange *change)
+{
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_by_table";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_by_table_cb(ctx, relation, change);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index f9ed1083df..54a04b54e7 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -57,6 +57,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_table_filter(struct LogicalDecodingContext *ctx,
+								  Relation relation,
+								  ReorderBufferChange *change);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -259,6 +262,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_by_table_cb = pgoutput_table_filter;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1679,6 +1683,57 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 	return false;
 }
 
+/*
+ * Return true if the relation has not been published, false otherwise.
+ */
+static bool
+pgoutput_table_filter(struct LogicalDecodingContext *ctx,
+					  Relation relation,
+					  ReorderBufferChange *change)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *relentry;
+	ReorderBufferChangeType action = change->action;
+
+	if (!is_publishable_relation(relation))
+		return true;
+
+	relentry = get_rel_sync_entry(data, relation);
+
+	/* First check the table filter */
+	switch (action)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			if (!relentry->pubactions.pubinsert)
+				return true;
+			break;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			if (!relentry->pubactions.pubupdate)
+				return true;
+			break;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			if (!relentry->pubactions.pubdelete)
+				return true;
+
+			/*
+			 * This is only possible if deletes are allowed even when replica
+			 * identity is not defined for a table. Since the DELETE action
+			 * can't be published, we simply return.
+			 */
+			if (!change->data.tp.oldtuple)
+			{
+				elog(DEBUG1, "didn't send DELETE change because of missing oldtuple");
+				return true;
+			}
+			break;
+		default:
+			Assert(false);
+	}
+
+	return false;
+}
+
+
 /*
  * Shutdown the output plugin.
  *
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index dffc0d1564..452425ba5c 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -145,6 +145,8 @@ extern void LogicalConfirmReceivedLocation(XLogRecPtr lsn);
 extern bool filter_prepare_cb_wrapper(LogicalDecodingContext *ctx,
 									  TransactionId xid, const char *gid);
 extern bool filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id);
+extern bool filter_by_table_cb_wrapper(LogicalDecodingContext *ctx, Relation relation, 
+										ReorderBufferChange *change);
 extern void ResetLogicalStreamingState(void);
 extern void UpdateDecodingStats(LogicalDecodingContext *ctx);
 
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 2ffcf17505..4c885639d7 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -96,6 +96,13 @@ typedef void (*LogicalDecodeMessageCB) (struct LogicalDecodingContext *ctx,
 typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ctx,
 											   RepOriginId origin_id);
 
+/*
+ * Filter changes by table.
+ */
+typedef bool (*LogicalDecodeFilterByRelCB) (struct LogicalDecodingContext *ctx,
+											Relation relation,
+									  		ReorderBufferChange *change);
+
 /*
  * Called to shutdown an output plugin.
  */
@@ -222,6 +229,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterByRelCB filter_by_table_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/test/subscription/t/034_table_filter.pl b/src/test/subscription/t/034_table_filter.pl
new file mode 100644
index 0000000000..8d4f962adc
--- /dev/null
+++ b/src/test/subscription/t/034_table_filter.pl
@@ -0,0 +1,91 @@
+# Copyright (c) 2021-2023, PostgreSQL Global Development Group
+
+# Basic logical replication test
+use strict;
+use warnings;
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+# Initialize publisher node
+my $node_publisher = PostgreSQL::Test::Cluster->new('publisher');
+$node_publisher->init(allows_streaming => 'logical');
+$node_publisher->append_conf('postgresql.conf', 'logical_decoding_work_mem = 64kB');
+$node_publisher->start;
+
+# Create subscriber node
+my $node_subscriber = PostgreSQL::Test::Cluster->new('subscriber');
+$node_subscriber->init;
+$node_subscriber->start;
+
+
+# Create some preexisting content on publisher
+$node_publisher->safe_psql('postgres',
+	"create table tbl_pub(id int, val1 text, val2 text,size int);");
+$node_publisher->safe_psql('postgres',
+	"create table tbl_t1(id int, val1 text, val2 text,size int);");
+$node_publisher->safe_psql('postgres',
+	"CREATE PUBLICATION mypub FOR TABLE public.tbl_pub;");
+$node_publisher->safe_psql('postgres',
+qq(
+CREATE OR REPLACE FUNCTION check_replication_status() RETURNS VOID AS \$\$
+DECLARE
+    replication_record pg_stat_replication;
+BEGIN
+    LOOP
+        SELECT *
+        INTO replication_record
+        FROM pg_stat_replication
+        WHERE application_name = 'mysub';
+        
+        IF replication_record.replay_lsn = replication_record.write_lsn THEN
+            EXIT;
+        END IF;
+    
+        PERFORM pg_sleep(1);
+    END LOOP;
+END;
+\$\$ LANGUAGE plpgsql;));
+
+# Create some preexisting content on subscriber
+my $publisher_connstr = $node_publisher->connstr . ' dbname=postgres';
+$node_subscriber->safe_psql('postgres', 
+    "create table tbl_pub(id int, val1 text, val2 text,size text);");
+$node_subscriber->safe_psql('postgres',
+    "create table tbl_t1(id int, val1 text, val2 text,size text);");
+$node_subscriber->safe_psql('postgres',
+	"CREATE SUBSCRIPTION mysub CONNECTION '$publisher_connstr' PUBLICATION mypub"
+);
+
+# Wait for initial table sync to finish
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'mysub');
+
+# test filter
+$node_publisher->safe_psql('postgres',
+qq(BEGIN;
+insert into tbl_t1 select 1, 'xyzzy', 'abcba', sum(size) from pg_ls_replslotdir('mysub');
+insert into tbl_t1 select i,repeat('xyzzy', i),repeat('abcba',i),(select sum(size) from pg_ls_replslotdir('mysub')) from generate_series(2,9999) i;
+update tbl_t1 set val2 = repeat('xyzzy',id) where id > 1 and id < 10001;
+select check_replication_status();
+insert into tbl_t1 select 10001, 'xyzzy', 'abcba', sum(size) from pg_ls_replslotdir('mysub');
+COMMIT;)
+);
+
+my $minsize =   $node_publisher->safe_psql('postgres',
+	"select size from tbl_t1 order by size asc limit 1;");
+my $maxsize =   $node_publisher->safe_psql('postgres',
+	"select size from tbl_t1 order by size desc limit 1;");
+is($minsize, $maxsize, 'check decode filter table between maxsize and minsize');
+
+
+my $fristrow =   $node_publisher->safe_psql('postgres',
+	"select size from tbl_t1 where id = 1;");
+my $lastrow =   $node_publisher->safe_psql('postgres',
+	"select size from tbl_t1 where id = 10001;");
+is($minsize, $maxsize, 'check decode filter table between fristrow and lastrow');
+
+is($minsize, $lastrow, 'check decode filter table between minsize and lastrow');
+
+print "minsize: " . $minsize . "maxsize: " . $maxsize ."fristrow: " . $fristrow ."lastrow: " . $lastrow . "\n";
+
+done_testing();
\ No newline at end of file
-- 
2.39.3

#3Amit Kapila
amit.kapila16@gmail.com
In reply to: li jie (#2)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Fri, Dec 1, 2023 at 1:55 PM li jie <ggysxcq@gmail.com> wrote:

This is just an immature idea. I haven't started to implement it yet.
Maybe it was designed this way because there
are key factors that I didn't consider. So I want to hear everyone's
opinions, especially the designers of logic decoding.

Attached is the patch I used to implement this optimization.
The main designs are as follows:
1. Added a callback LogicalDecodeFilterByRelCB for the output plugin.

2. Added this callback function pgoutput_table_filter for the pgoutput plugin.
Its main implementation is based on the table filter in the
pgoutput_change function.

3. After constructing a change and before Queue a change into a transaction,
use RelidByRelfilenumber to obtain the relation associated with the change,
just like obtaining the relation in the ReorderBufferProcessTXN function.

4. Relation may be a toast, and there is no good way to get its real
table relation
based on toast relation. Here, I get the real table oid through
toast relname, and
then get the real table relation.

This may be helpful for the case you have mentioned but how about
cases where there is nothing to filter by relation? It will add
overhead related to the transaction start/end and others for each
change. Currently, we do that just once for all the changes that need
to be processed. I wonder why the spilling can't be avoided with GUC
logical_decoding_work_mem?

--
With Regards,
Amit Kapila.

#4li jie
ggysxcq@gmail.com
In reply to: Amit Kapila (#3)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

This may be helpful for the case you have mentioned but how about
cases where there is nothing to filter by relation?

You can see the complete antecedent in the email [1]. Relation that has
not been published will also generate changes and put them into the entire
transaction group, which will increase invalid memory or disk space.

It will add
overhead related to the transaction start/end and others for each
change. Currently, we do that just once for all the changes that need
to be processed.

Yes, it will only be processed once at present. It is done when applying
each change when the transaction is committed. This patch hopes to
advance it to the time when constructing the change, and determines the
change queue into a based on whether the relation is published.

I wonder why the spilling can't be avoided with GUC
logical_decoding_work_mem?

Of course you can, but this will only convert disk space into memory space.
For details, please see the case in Email [1].

Regards, lijie

#5li jie
ggysxcq@gmail.com
In reply to: Amit Kapila (#3)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

This may be helpful for the case you have mentioned but how about
cases where there is nothing to filter by relation?

You can see the complete antecedent in the email [1]/messages/by-id/CAGfChW51P944nM5h0HTV9HistvVfwBxNaMt_s-OZ9t=uXz+Zbg@mail.gmail.com. Relation that has
not been published will also generate changes and put them into the entire
transaction group, which will increase invalid memory or disk space.

It will add
overhead related to the transaction start/end and others for each
change. Currently, we do that just once for all the changes that need
to be processed.

Yes, it will only be processed once at present. It is done when applying
each change when the transaction is committed. This patch hopes to
advance it to the time when constructing the change, and determines the
change queue into a based on whether the relation is published.

I wonder why the spilling can't be avoided with GUC
logical_decoding_work_mem?

Of course you can, but this will only convert disk space into memory space.
For details, please see the case in Email [1]/messages/by-id/CAGfChW51P944nM5h0HTV9HistvVfwBxNaMt_s-OZ9t=uXz+Zbg@mail.gmail.com.

[1]: /messages/by-id/CAGfChW51P944nM5h0HTV9HistvVfwBxNaMt_s-OZ9t=uXz+Zbg@mail.gmail.com

Regards, lijie

Amit Kapila <amit.kapila16@gmail.com> 于2023年12月2日周六 12:11写道:

Show quoted text

On Fri, Dec 1, 2023 at 1:55 PM li jie <ggysxcq@gmail.com> wrote:

This is just an immature idea. I haven't started to implement it yet.
Maybe it was designed this way because there
are key factors that I didn't consider. So I want to hear everyone's
opinions, especially the designers of logic decoding.

Attached is the patch I used to implement this optimization.
The main designs are as follows:
1. Added a callback LogicalDecodeFilterByRelCB for the output plugin.

2. Added this callback function pgoutput_table_filter for the pgoutput plugin.
Its main implementation is based on the table filter in the
pgoutput_change function.

3. After constructing a change and before Queue a change into a transaction,
use RelidByRelfilenumber to obtain the relation associated with the change,
just like obtaining the relation in the ReorderBufferProcessTXN function.

4. Relation may be a toast, and there is no good way to get its real
table relation
based on toast relation. Here, I get the real table oid through
toast relname, and
then get the real table relation.

This may be helpful for the case you have mentioned but how about
cases where there is nothing to filter by relation? It will add
overhead related to the transaction start/end and others for each
change. Currently, we do that just once for all the changes that need
to be processed. I wonder why the spilling can't be avoided with GUC
logical_decoding_work_mem?

--
With Regards,
Amit Kapila.

#6Ajin Cherian
itsajin@gmail.com
In reply to: li jie (#5)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Of course you can, but this will only convert disk space into memory space.
For details, please see the case in Email [1].

[1]
/messages/by-id/CAGfChW51P944nM5h0HTV9HistvVfwBxNaMt_s-OZ9t=uXz+Zbg@mail.gmail.com

Regards, lijie

Hi lijie,

Overall, I think the patch is a good improvement. Some comments from first
run through of patch:
1. The patch no longer applies cleanly, please rebase.

2. While testing the patch, I saw something strange. If I try to truncate a
table that is published. I still see the message:
2024-03-18 22:25:51.243 EDT [29385] LOG: logical filter change by table
pg_class

This gives the impression that the truncate operation on the published
table has been filtered but it hasn't. Also the log message needs to be
reworded. Maybe, "Logical filtering change by non-published table
<relation_name>"

3. Below code:
@@ -1201,11 +1343,14 @@ DecodeMultiInsert(LogicalDecodingContext *ctx,
XLogRecordBuffer *buf)
+
+ if (FilterByTable(ctx, change))
+ continue;;

extra semi-colon after continue.

4. I am not sure if this is possible, but is there a way to avoid the
overhead in the patch if the publication publishes "ALL TABLES"?

5. In function: pgoutput_table_filter() - this code appears to be filtering
out not just unpublished tables but also applying row based filters on
published tables as well. Is this really within the scope of the feature?

regards,
Ajin Cherian
Fujitsu Australia

#7Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: li jie (#2)
2 attachment(s)
RE: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Dear Li,

Thanks for proposing and sharing the PoC. Here are my high-level comments.

1.
What if ALTER PUBLICATION ... ADD is executed in parallel?
Should we publish added tables if the altering is done before the transaction is
committed? The current patch seems unable to do so because changes for added
tables have not been queued at COMMIT.
If we should not publish such tables, why?

2.
This patch could not apply as-is. Please rebase.

3. FilterByTable()

```
+    if (ctx->callbacks.filter_by_origin_cb == NULL)
+        return false;
```

filter_by_table_cb should be checked instead of filter_by_origin_cb.
Current patch crashes if the filter_by_table_cb() is not implemented.

4. DecodeSpecConfirm()

```
+    if (FilterByTable(ctx, change))
+        return;
+
```

I'm not sure it is needed. Can you explain the reason why you added?

5. FilterByTable

```
+    switch (change->action)
+    {
+            /* intentionally fall through */
+        case REORDER_BUFFER_CHANGE_INSERT:
+        case REORDER_BUFFER_CHANGE_UPDATE:
+        case REORDER_BUFFER_CHANGE_DELETE:
+            break;
+        default:
+            return false;
+    }
```

IIUC, REORDER_BUFFER_CHANGE_TRUNCATE also targes the user table, so I think
it should be accepted. Thought?

6.

I got strange errors when I tested the feature. I thought this implied there were
bugs in your patch.

1. implemented no-op filter atop test_decoding like attached
2. ran `make check` for test_decoding modle
3. some tests failed. Note that "filter" was a test added by me.
regression.diffs was also attached.

```
not ok 1 - ddl 970 ms
ok 2 - xact 36 ms
not ok 3 - rewrite 525 ms
not ok 4 - toast 736 ms
ok 5 - permissions 50 ms
ok 6 - decoding_in_xact 39 ms
not ok 7 - decoding_into_rel 57 ms
ok 8 - binary 21 ms
not ok 9 - prepared 33 ms
ok 10 - replorigin 93 ms
ok 11 - time 25 ms
ok 12 - messages 47 ms
ok 13 - spill 8063 ms
ok 14 - slot 124 ms
ok 15 - truncate 37 ms
not ok 16 - stream 60 ms
ok 17 - stats 157 ms
ok 18 - twophase 122 ms
not ok 19 - twophase_stream 57 ms
not ok 20 - filter 20 ms
```

Currently I'm not 100% sure the reason, but I think it may read the latest system
catalog even if ALTER SUBSCRIPTION is executed after changes.
In below example, an attribute is altered text->somenum, after inserting data.
But get_changes() outputs as somenum.

```
  BEGIN
- table public.replication_example: INSERT: id[integer]:1 somedata[integer]:1 text[character varying]:'1'
- table public.replication_example: INSERT: id[integer]:2 somedata[integer]:1 text[character varying]:'2'
+ table public.replication_example: INSERT: id[integer]:1 somedata[integer]:1 somenum[character varying]:'1'
+ table public.replication_example: INSERT: id[integer]:2 somedata[integer]:1 somenum[character varying]:'2'
  COMMIT
```

Also, if the relfilenuber of the relation is changed, an ERROR is raised.

```
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                    data                                    
-----------------------------------------------------------------------------
- BEGIN
- table public.tr_pkey: INSERT: id2[integer]:2 data[integer]:1 id[integer]:2
- COMMIT
- BEGIN
- table public.tr_pkey: DELETE: id[integer]:1
- table public.tr_pkey: DELETE: id[integer]:2
- COMMIT
-(7 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
```

Best Regards,
Hayato Kuroda
FUJITSU LIMITED
https://www.fujitsu.com/

Attachments:

0001-WIP-implement-no-op-filter.txttext/plain; name=0001-WIP-implement-no-op-filter.txtDownload
From 9bfb34dad67745f1155238a4b11bbe1d28284b50 Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Mon, 20 May 2024 03:19:56 +0000
Subject: [PATCH] WIP: implement no-op filter

---
 contrib/test_decoding/Makefile            |  2 +-
 contrib/test_decoding/expected/filter.out |  0
 contrib/test_decoding/meson.build         |  1 +
 contrib/test_decoding/sql/filter.sql      | 13 +++++++++++++
 contrib/test_decoding/test_decoding.c     | 18 ++++++++++++++++++
 src/backend/replication/logical/decode.c  |  2 +-
 6 files changed, 34 insertions(+), 2 deletions(-)
 create mode 100644 contrib/test_decoding/expected/filter.out
 create mode 100644 contrib/test_decoding/sql/filter.sql

diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index c7ce603706..b9b6034b9f 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -5,7 +5,7 @@ PGFILEDESC = "test_decoding - example of a logical decoding output plugin"
 
 REGRESS = ddl xact rewrite toast permissions decoding_in_xact \
 	decoding_into_rel binary prepared replorigin time messages \
-	spill slot truncate stream stats twophase twophase_stream
+	spill slot truncate stream stats twophase twophase_stream filter
 ISOLATION = mxact delayed_startup ondisk_startup concurrent_ddl_dml \
 	oldest_xmin snapshot_transfer subxact_without_top concurrent_stream \
 	twophase_snapshot slot_creation_error catalog_change_snapshot
diff --git a/contrib/test_decoding/expected/filter.out b/contrib/test_decoding/expected/filter.out
new file mode 100644
index 0000000000..e69de29bb2
diff --git a/contrib/test_decoding/meson.build b/contrib/test_decoding/meson.build
index f1548c0faf..539a148708 100644
--- a/contrib/test_decoding/meson.build
+++ b/contrib/test_decoding/meson.build
@@ -41,6 +41,7 @@ tests += {
       'stats',
       'twophase',
       'twophase_stream',
+      'filter',
     ],
     'regress_args': [
       '--temp-config', files('logical.conf'),
diff --git a/contrib/test_decoding/sql/filter.sql b/contrib/test_decoding/sql/filter.sql
new file mode 100644
index 0000000000..4cd4f1d824
--- /dev/null
+++ b/contrib/test_decoding/sql/filter.sql
@@ -0,0 +1,13 @@
+-- predictability
+SET synchronous_commit = on;
+
+-- define table which will be filtered
+CREATE TABLE test_tobeskipped (a int);
+
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+INSERT INTO test_tobeskipped VALUES (generate_series(1, 10));
+
+-- check that changes to test_tobeskipped are not output
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 7c50d13969..6f0d076305 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -68,6 +68,9 @@ static void pg_decode_truncate(LogicalDecodingContext *ctx,
 							   ReorderBufferChange *change);
 static bool pg_decode_filter(LogicalDecodingContext *ctx,
 							 RepOriginId origin_id);
+static bool pg_decode_filter_table(struct LogicalDecodingContext *ctx,
+								   Relation relation,
+								   ReorderBufferChange *change);
 static void pg_decode_message(LogicalDecodingContext *ctx,
 							  ReorderBufferTXN *txn, XLogRecPtr lsn,
 							  bool transactional, const char *prefix,
@@ -133,6 +136,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->truncate_cb = pg_decode_truncate;
 	cb->commit_cb = pg_decode_commit_txn;
 	cb->filter_by_origin_cb = pg_decode_filter;
+	cb->filter_by_table_cb = pg_decode_filter_table;
 	cb->shutdown_cb = pg_decode_shutdown;
 	cb->message_cb = pg_decode_message;
 	cb->filter_prepare_cb = pg_decode_filter_prepare;
@@ -467,6 +471,20 @@ pg_decode_filter(LogicalDecodingContext *ctx,
 	return false;
 }
 
+/*
+ * Filter out by the table.
+ *
+ * It is implemented for demonstating and testing purpose. If the table
+ * contains the "_tobeskipped" subsctring, then we filter out.
+ */
+static bool
+pg_decode_filter_table(struct LogicalDecodingContext *ctx,
+					   Relation relation,
+					   ReorderBufferChange *change)
+{
+	return false;
+}
+
 /*
  * Print literal `outputstr' already represented as string of type `typid'
  * into stringbuf `s'.
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index f2b2e33521..4266a9b353 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -589,7 +589,7 @@ FilterByTable(LogicalDecodingContext *ctx, ReorderBufferChange *change)
 	bool		result = false;
 	bool		using_subtxn;
 
-	if (ctx->callbacks.filter_by_origin_cb == NULL)
+	if (ctx->callbacks.filter_by_table_cb == NULL)
 		return false;
 
 	switch (change->action)
-- 
2.43.0

regression.diffsapplication/octet-stream; name=regression.diffsDownload
diff -U3 /home/hayato/patchTest/postgres/contrib/test_decoding/expected/ddl.out /home/hayato/patchTest/postgres/contrib/test_decoding/results/ddl.out
--- /home/hayato/patchTest/postgres/contrib/test_decoding/expected/ddl.out	2024-05-14 00:12:11.528895957 +0000
+++ /home/hayato/patchTest/postgres/contrib/test_decoding/results/ddl.out	2024-05-20 04:49:00.215181926 +0000
@@ -90,26 +90,26 @@
 INSERT INTO replication_example(somedata, somenum) VALUES (4, 1);
 -- collect all changes
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                                           data                                                            
----------------------------------------------------------------------------------------------------------------------------
+                                                    data                                                     
+-------------------------------------------------------------------------------------------------------------
  BEGIN
- table public.replication_example: INSERT: id[integer]:1 somedata[integer]:1 text[character varying]:'1'
- table public.replication_example: INSERT: id[integer]:2 somedata[integer]:1 text[character varying]:'2'
+ table public.replication_example: INSERT: id[integer]:1 somedata[integer]:1 somenum[character varying]:'1'
+ table public.replication_example: INSERT: id[integer]:2 somedata[integer]:1 somenum[character varying]:'2'
  COMMIT
  BEGIN
- table public.replication_example: INSERT: id[integer]:3 somedata[integer]:2 text[character varying]:'1' bar[integer]:4
+ table public.replication_example: INSERT: id[integer]:3 somedata[integer]:2 somenum[character varying]:'1'
  COMMIT
  BEGIN
- table public.replication_example: INSERT: id[integer]:4 somedata[integer]:2 text[character varying]:'2' bar[integer]:4
- table public.replication_example: INSERT: id[integer]:5 somedata[integer]:2 text[character varying]:'3' bar[integer]:4
- table public.replication_example: INSERT: id[integer]:6 somedata[integer]:2 text[character varying]:'4' bar[integer]:null
+ table public.replication_example: INSERT: id[integer]:4 somedata[integer]:2 somenum[character varying]:'2'
+ table public.replication_example: INSERT: id[integer]:5 somedata[integer]:2 somenum[character varying]:'3'
+ table public.replication_example: INSERT: id[integer]:6 somedata[integer]:2 somenum[character varying]:'4'
  COMMIT
  BEGIN
- table public.replication_example: INSERT: id[integer]:7 somedata[integer]:3 text[character varying]:'1'
+ table public.replication_example: INSERT: id[integer]:7 somedata[integer]:3 somenum[character varying]:'1'
  COMMIT
  BEGIN
- table public.replication_example: INSERT: id[integer]:8 somedata[integer]:3 text[character varying]:'2'
- table public.replication_example: INSERT: id[integer]:9 somedata[integer]:3 text[character varying]:'3'
+ table public.replication_example: INSERT: id[integer]:8 somedata[integer]:3 somenum[character varying]:'2'
+ table public.replication_example: INSERT: id[integer]:9 somedata[integer]:3 somenum[character varying]:'3'
  COMMIT
  BEGIN
  table public.replication_example: INSERT: id[integer]:10 somedata[integer]:4 somenum[character varying]:'1'
@@ -135,13 +135,13 @@
 COMMIT;
 -- show changes
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                                                   data                                                                   
-------------------------------------------------------------------------------------------------------------------------------------------
+                                                                    data                                                                     
+---------------------------------------------------------------------------------------------------------------------------------------------
  BEGIN
- table public.replication_example: INSERT: id[integer]:11 somedata[integer]:5 somenum[integer]:1
+ table public.replication_example: INSERT: id[integer]:11 somedata[integer]:5 somenum[integer]:1 zaphod1[integer]:null zaphod2[integer]:null
  COMMIT
  BEGIN
- table public.replication_example: INSERT: id[integer]:12 somedata[integer]:6 somenum[integer]:1
+ table public.replication_example: INSERT: id[integer]:12 somedata[integer]:6 somenum[integer]:1 zaphod1[integer]:null zaphod2[integer]:null
  table public.replication_example: INSERT: id[integer]:13 somedata[integer]:6 somenum[integer]:2 zaphod1[integer]:1
  table public.replication_example: INSERT: id[integer]:14 somedata[integer]:6 somenum[integer]:3 zaphod1[integer]:null zaphod2[integer]:1
  table public.replication_example: INSERT: id[integer]:15 somedata[integer]:6 somenum[integer]:4 zaphod1[integer]:2 zaphod2[integer]:null
@@ -243,32 +243,13 @@
 ALTER TABLE tr_unique RENAME TO tr_pkey;
 ALTER TABLE tr_pkey ADD COLUMN id serial primary key;
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1', 'include-rewrites', '1');
-                                    data                                     
------------------------------------------------------------------------------
- BEGIN
- table public.tr_unique: INSERT: id2[integer]:1 data[integer]:10
- COMMIT
- BEGIN
- table public.tr_pkey: INSERT: id2[integer]:1 data[integer]:10 id[integer]:1
- COMMIT
-(6 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 INSERT INTO tr_pkey(data) VALUES(1);
 --show deletion with primary key
 DELETE FROM tr_pkey;
 /* display results */
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                    data                                    
-----------------------------------------------------------------------------
- BEGIN
- table public.tr_pkey: INSERT: id2[integer]:2 data[integer]:1 id[integer]:2
- COMMIT
- BEGIN
- table public.tr_pkey: DELETE: id[integer]:1
- table public.tr_pkey: DELETE: id[integer]:2
- COMMIT
-(7 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 /*
  * check that disk spooling works (also for logical messages)
  */
@@ -291,15 +272,7 @@
 FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
 GROUP BY substring(data, 1, 24)
 ORDER BY 1,2;
- count |                                  min                                  |                                  max                                   
--------+-----------------------------------------------------------------------+------------------------------------------------------------------------
-     1 | BEGIN                                                                 | BEGIN
-     1 | COMMIT                                                                | COMMIT
-     1 | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg | message: transactional: 1 prefix: test, sz: 14 content:tx logical msg
-     1 | table public.tr_oddlength: INSERT: id[text]:'ab' data[text]:'foo'     | table public.tr_oddlength: INSERT: id[text]:'ab' data[text]:'foo'
- 20467 | table public.tr_etoomuch: DELETE: id[integer]:1                       | table public.tr_etoomuch: UPDATE: id[integer]:9999 data[integer]:-9999
-(5 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 -- check updates of primary keys work correctly
 BEGIN;
 CREATE TABLE spoolme AS SELECT g.i FROM generate_series(1, 5000) g(i);
@@ -312,13 +285,7 @@
 SELECT data
 FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
 WHERE data ~ 'UPDATE';
-                                                    data                                                     
--------------------------------------------------------------------------------------------------------------
- table public.tr_etoomuch: UPDATE: old-key: id[integer]:5000 new-tuple: id[integer]:-5000 data[integer]:5000
- table public.tr_oddlength: UPDATE: old-key: id[text]:'ab' new-tuple: id[text]:'x' data[text]:'quux'
- table public.tr_oddlength: UPDATE: old-key: id[text]:'x' new-tuple: id[text]:'yy' data[text]:'a'
-(3 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 -- check that a large, spooled, upsert works
 INSERT INTO tr_etoomuch (id, data)
 SELECT g.i, -g.i FROM generate_series(8000, 12000) g(i)
@@ -327,14 +294,7 @@
 FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1') WITH ORDINALITY
 GROUP BY 1
 ORDER BY min(ordinality);
-           substring           | count 
--------------------------------+-------
- BEGIN                         |     1
- table public.tr_etoomuch: UPD |  2235
- table public.tr_etoomuch: INS |  1766
- COMMIT                        |     1
-(4 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 /*
  * check whether we decode subtransactions correctly in relation with each
  * other
@@ -356,18 +316,7 @@
 RELEASE SAVEPOINT b;
 COMMIT;
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                 data                                 
-----------------------------------------------------------------------
- BEGIN
- table public.tr_sub: INSERT: id[integer]:1 path[text]:'1-top-#1'
- table public.tr_sub: INSERT: id[integer]:2 path[text]:'1-top-1-#1'
- table public.tr_sub: INSERT: id[integer]:3 path[text]:'1-top-1-#2'
- table public.tr_sub: INSERT: id[integer]:4 path[text]:'1-top-2-1-#1'
- table public.tr_sub: INSERT: id[integer]:5 path[text]:'1-top-2-1-#2'
- table public.tr_sub: INSERT: id[integer]:6 path[text]:'1-top-2-#1'
- COMMIT
-(8 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 -- check that we handle xlog assignments correctly
 BEGIN;
 -- nest 80 subtxns
@@ -395,16 +344,7 @@
 INSERT INTO tr_sub(path) VALUES ('2-top-#1');
 COMMIT;
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                  data                                  
-------------------------------------------------------------------------
- BEGIN
- table public.tr_sub: INSERT: id[integer]:7 path[text]:'2-top-1...--#1'
- table public.tr_sub: INSERT: id[integer]:8 path[text]:'2-top-1...--#2'
- table public.tr_sub: INSERT: id[integer]:9 path[text]:'2-top-1...--#3'
- table public.tr_sub: INSERT: id[integer]:10 path[text]:'2-top-#1'
- COMMIT
-(6 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 -- make sure rollbacked subtransactions aren't decoded
 BEGIN;
 INSERT INTO tr_sub(path) VALUES ('3-top-2-#1');
@@ -416,15 +356,7 @@
 INSERT INTO tr_sub(path) VALUES ('3-top-2-#2');
 COMMIT;
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                 data                                  
------------------------------------------------------------------------
- BEGIN
- table public.tr_sub: INSERT: id[integer]:11 path[text]:'3-top-2-#1'
- table public.tr_sub: INSERT: id[integer]:12 path[text]:'3-top-2-1-#1'
- table public.tr_sub: INSERT: id[integer]:14 path[text]:'3-top-2-#2'
- COMMIT
-(5 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 -- test whether a known, but not yet logged toplevel xact, followed by a
 -- subxact commit is handled correctly
 BEGIN;
@@ -445,16 +377,7 @@
 INSERT INTO tr_sub(path) VALUES ('5-top-1-#1');
 COMMIT;
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                data                                 
----------------------------------------------------------------------
- BEGIN
- table public.tr_sub: INSERT: id[integer]:15 path[text]:'4-top-1-#1'
- COMMIT
- BEGIN
- table public.tr_sub: INSERT: id[integer]:16 path[text]:'5-top-1-#1'
- COMMIT
-(6 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 -- check that DDL in aborted subtransactions handled correctly
 CREATE TABLE tr_sub_ddl(data int);
 BEGIN;
@@ -466,13 +389,7 @@
 INSERT INTO tr_sub_ddl VALUES(43);
 COMMIT;
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                       data                       
---------------------------------------------------
- BEGIN
- table public.tr_sub_ddl: INSERT: data[bigint]:43
- COMMIT
-(3 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 /*
  * Check whether treating a table as a catalog table works somewhat
  */
@@ -543,22 +460,7 @@
 INSERT INTO replication_metadata(relation, options)
 VALUES ('zaphod', NULL);
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                                                data                                                                
-------------------------------------------------------------------------------------------------------------------------------------
- BEGIN
- table public.replication_metadata: INSERT: id[integer]:1 relation[name]:'foo' options[text[]]:'{a,b}'
- COMMIT
- BEGIN
- table public.replication_metadata: INSERT: id[integer]:2 relation[name]:'bar' options[text[]]:'{a,b}'
- COMMIT
- BEGIN
- table public.replication_metadata: INSERT: id[integer]:3 relation[name]:'blub' options[text[]]:null
- COMMIT
- BEGIN
- table public.replication_metadata: INSERT: id[integer]:4 relation[name]:'zaphod' options[text[]]:null rewritemeornot[integer]:null
- COMMIT
-(12 rows)
-
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 /*
  * check whether we handle updates/deletes correct with & without a pkey
  */
@@ -663,151 +565,7 @@
 -- produce 200kB of useless dashes. Turn that off temporarily to avoid it.
 \pset format unaligned
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-data
-BEGIN
-table public.table_without_key: INSERT: id[integer]:1 data[integer]:1
-table public.table_without_key: INSERT: id[integer]:2 data[integer]:2
-COMMIT
-BEGIN
-table public.table_without_key: DELETE: (no-tuple-data)
-COMMIT
-BEGIN
-table public.table_without_key: UPDATE: id[integer]:2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_without_key: UPDATE: id[integer]:-2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_without_key: UPDATE: id[integer]:2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_without_key: UPDATE: old-key: id[integer]:2 data[integer]:3 new-tuple: id[integer]:-2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_without_key: UPDATE: old-key: id[integer]:-2 data[integer]:3 new-tuple: id[integer]:2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_without_key: UPDATE: old-key: id[integer]:2 data[integer]:3 new-tuple: id[integer]:-2 data[integer]:3 new_column[text]:null
-COMMIT
-BEGIN
-table public.table_without_key: UPDATE: old-key: id[integer]:-2 data[integer]:3 new-tuple: id[integer]:2 data[integer]:3 new_column[text]:'someval'
-COMMIT
-BEGIN
-table public.table_without_key: DELETE: id[integer]:2 data[integer]:3 new_column[text]:'someval'
-COMMIT
-BEGIN
-table public.table_with_pkey: INSERT: id[integer]:1 data[integer]:1
-table public.table_with_pkey: INSERT: id[integer]:2 data[integer]:2
-COMMIT
-BEGIN
-table public.table_with_pkey: DELETE: id[integer]:1
-COMMIT
-BEGIN
-table public.table_with_pkey: UPDATE: id[integer]:2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_pkey: UPDATE: old-key: id[integer]:2 new-tuple: id[integer]:-2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_pkey: UPDATE: old-key: id[integer]:-2 new-tuple: id[integer]:2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_pkey: UPDATE: old-key: id[integer]:2 new-tuple: id[integer]:-2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_pkey: UPDATE: old-key: id[integer]:-2 new-tuple: id[integer]:2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_pkey: DELETE: id[integer]:2
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: INSERT: id[integer]:1 data[integer]:1
-table public.table_with_unique_not_null: INSERT: id[integer]:2 data[integer]:2
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: DELETE: (no-tuple-data)
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: UPDATE: id[integer]:2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: UPDATE: id[integer]:-2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: UPDATE: id[integer]:2 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: DELETE: (no-tuple-data)
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: INSERT: id[integer]:3 data[integer]:1
-table public.table_with_unique_not_null: INSERT: id[integer]:4 data[integer]:2
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: DELETE: id[integer]:3
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: UPDATE: id[integer]:4 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: UPDATE: old-key: id[integer]:4 new-tuple: id[integer]:-4 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: UPDATE: old-key: id[integer]:-4 new-tuple: id[integer]:4 data[integer]:3
-COMMIT
-BEGIN
-table public.table_with_unique_not_null: DELETE: id[integer]:4
-COMMIT
-BEGIN
-table public.table_dropped_index_with_pk: INSERT: a[integer]:1 b[integer]:1 c[integer]:1
-table public.table_dropped_index_with_pk: INSERT: a[integer]:2 b[integer]:2 c[integer]:2
-table public.table_dropped_index_with_pk: INSERT: a[integer]:3 b[integer]:3 c[integer]:3
-COMMIT
-BEGIN
-table public.table_dropped_index_with_pk: UPDATE: a[integer]:4 b[integer]:1 c[integer]:1
-COMMIT
-BEGIN
-table public.table_dropped_index_with_pk: UPDATE: a[integer]:2 b[integer]:5 c[integer]:2
-COMMIT
-BEGIN
-table public.table_dropped_index_with_pk: UPDATE: a[integer]:3 b[integer]:6 c[integer]:7
-COMMIT
-BEGIN
-table public.table_dropped_index_with_pk: DELETE: (no-tuple-data)
-COMMIT
-BEGIN
-table public.table_dropped_index_with_pk: DELETE: (no-tuple-data)
-COMMIT
-BEGIN
-table public.table_dropped_index_no_pk: INSERT: a[integer]:1 b[integer]:1 c[integer]:1
-table public.table_dropped_index_no_pk: INSERT: a[integer]:2 b[integer]:2 c[integer]:2
-table public.table_dropped_index_no_pk: INSERT: a[integer]:3 b[integer]:3 c[integer]:3
-COMMIT
-BEGIN
-table public.table_dropped_index_no_pk: UPDATE: a[integer]:4 b[integer]:1 c[integer]:1
-COMMIT
-BEGIN
-table public.table_dropped_index_no_pk: UPDATE: a[integer]:2 b[integer]:5 c[integer]:2
-COMMIT
-BEGIN
-table public.table_dropped_index_no_pk: UPDATE: a[integer]:3 b[integer]:6 c[integer]:7
-COMMIT
-BEGIN
-table public.table_dropped_index_no_pk: DELETE: (no-tuple-data)
-COMMIT
-BEGIN
-table public.table_dropped_index_no_pk: DELETE: (no-tuple-data)
-COMMIT
-BEGIN
-table public.toasttable: INSERT: id[integer]:1 toasted_col1[text]:'12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361136213631364136513661367136813691370137113721373137413751376137713781379138013811382138313841385138613871388138913901391139213931394139513961397139813991400140114021403140414051406140714081409141014111412141314141415141614171418141914201421142214231424142514261427142814291430143114321433143414351436143714381439144014411442144314441445144614471448144914501451145214531454145514561457145814591460146114621463146414651466146714681469147014711472147314741475147614771478147914801481148214831484148514861487148814891490149114921493149414951496149714981499150015011502150315041505150615071508150915101511151215131514151515161517151815191520152115221523152415251526152715281529153015311532153315341535153615371538153915401541154215431544154515461547154815491550155115521553155415551556155715581559156015611562156315641565156615671568156915701571157215731574157515761577157815791580158115821583158415851586158715881589159015911592159315941595159615971598159916001601160216031604160516061607160816091610161116121613161416151616161716181619162016211622162316241625162616271628162916301631163216331634163516361637163816391640164116421643164416451646164716481649165016511652165316541655165616571658165916601661166216631664166516661667166816691670167116721673167416751676167716781679168016811682168316841685168616871688168916901691169216931694169516961697169816991700170117021703170417051706170717081709171017111712171317141715171617171718171917201721172217231724172517261727172817291730173117321733173417351736173717381739174017411742174317441745174617471748174917501751175217531754175517561757175817591760176117621763176417651766176717681769177017711772177317741775177617771778177917801781178217831784178517861787178817891790179117921793179417951796179717981799180018011802180318041805180618071808180918101811181218131814181518161817181818191820182118221823182418251826182718281829183018311832183318341835183618371838183918401841184218431844184518461847184818491850185118521853185418551856185718581859186018611862186318641865186618671868186918701871187218731874187518761877187818791880188118821883188418851886188718881889189018911892189318941895189618971898189919001901190219031904190519061907190819091910191119121913191419151916191719181919192019211922192319241925192619271928192919301931193219331934193519361937193819391940194119421943194419451946194719481949195019511952195319541955195619571958195919601961196219631964196519661967196819691970197119721973197419751976197719781979198019811982198319841985198619871988198919901991199219931994199519961997199819992000' rand1[double precision]:79 toasted_col2[text]:null rand2[double precision]:1578
-COMMIT
-BEGIN
-table public.toasttable: INSERT: id[integer]:2 toasted_col1[text]:null rand1[double precision]:3077 toasted_col2[text]:'0001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500000100020003000400050006000700080009001000110012001300140015001600170018001900200021002200230024002500260027002800290030003100320033003400350036003700380039004000410042004300440045004600470048004900500051005200530054005500560057005800590060006100620063006400650066006700680069007000710072007300740075007600770078007900800081008200830084008500860087008800890090009100920093009400950096009700980099010001010102010301040105010601070108010901100111011201130114011501160117011801190120012101220123012401250126012701280129013001310132013301340135013601370138013901400141014201430144014501460147014801490150015101520153015401550156015701580159016001610162016301640165016601670168016901700171017201730174017501760177017801790180018101820183018401850186018701880189019001910192019301940195019601970198019902000201020202030204020502060207020802090210021102120213021402150216021702180219022002210222022302240225022602270228022902300231023202330234023502360237023802390240024102420243024402450246024702480249025002510252025302540255025602570258025902600261026202630264026502660267026802690270027102720273027402750276027702780279028002810282028302840285028602870288028902900291029202930294029502960297029802990300030103020303030403050306030703080309031003110312031303140315031603170318031903200321032203230324032503260327032803290330033103320333033403350336033703380339034003410342034303440345034603470348034903500351035203530354035503560357035803590360036103620363036403650366036703680369037003710372037303740375037603770378037903800381038203830384038503860387038803890390039103920393039403950396039703980399040004010402040304040405040604070408040904100411041204130414041504160417041804190420042104220423042404250426042704280429043004310432043304340435043604370438043904400441044204430444044504460447044804490450045104520453045404550456045704580459046004610462046304640465046604670468046904700471047204730474047504760477047804790480048104820483048404850486048704880489049004910492049304940495049604970498049905000001000200030004000500060007000800090010001100120013001400150016001700180019002000210022002300240025002600270028002900300031003200330034003500360037003800390040004100420043004400450046004700480049005000510052005300540055005600570058005900600061006200630064006500660067006800690070007100720073007400750076007700780079008000810082008300840085008600870088008900900091009200930094009500960097009800990100010101020103010401050106010701080109011001110112011301140115011601170118011901200121012201230124012501260127012801290130013101320133013401350136013701380139014001410142014301440145014601470148014901500151015201530154015501560157015801590160016101620163016401650166016701680169017001710172017301740175017601770178017901800181018201830184018501860187018801890190019101920193019401950196019701980199020002010202020302040205020602070208020902100211021202130214021502160217021802190220022102220223022402250226022702280229023002310232023302340235023602370238023902400241024202430244024502460247024802490250025102520253025402550256025702580259026002610262026302640265026602670268026902700271027202730274027502760277027802790280028102820283028402850286028702880289029002910292029302940295029602970298029903000301030203030304030503060307030803090310031103120313031403150316031703180319032003210322032303240325032603270328032903300331033203330334033503360337033803390340034103420343034403450346034703480349035003510352035303540355035603570358035903600361036203630364036503660367036803690370037103720373037403750376037703780379038003810382038303840385038603870388038903900391039203930394039503960397039803990400040104020403040404050406040704080409041004110412041304140415041604170418041904200421042204230424042504260427042804290430043104320433043404350436043704380439044004410442044304440445044604470448044904500451045204530454045504560457045804590460046104620463046404650466046704680469047004710472047304740475047604770478047904800481048204830484048504860487048804890490049104920493049404950496049704980499050000010002000300040005000600070008000900100011001200130014001500160017001800190020002100220023002400250026002700280029003000310032003300340035003600370038003900400041004200430044004500460047004800490050005100520053005400550056005700580059006000610062006300640065006600670068006900700071007200730074007500760077007800790080008100820083008400850086008700880089009000910092009300940095009600970098009901000101010201030104010501060107010801090110011101120113011401150116011701180119012001210122012301240125012601270128012901300131013201330134013501360137013801390140014101420143014401450146014701480149015001510152015301540155015601570158015901600161016201630164016501660167016801690170017101720173017401750176017701780179018001810182018301840185018601870188018901900191019201930194019501960197019801990200020102020203020402050206020702080209021002110212021302140215021602170218021902200221022202230224022502260227022802290230023102320233023402350236023702380239024002410242024302440245024602470248024902500251025202530254025502560257025802590260026102620263026402650266026702680269027002710272027302740275027602770278027902800281028202830284028502860287028802890290029102920293029402950296029702980299030003010302030303040305030603070308030903100311031203130314031503160317031803190320032103220323032403250326032703280329033003310332033303340335033603370338033903400341034203430344034503460347034803490350035103520353035403550356035703580359036003610362036303640365036603670368036903700371037203730374037503760377037803790380038103820383038403850386038703880389039003910392039303940395039603970398039904000401040204030404040504060407040804090410041104120413041404150416041704180419042004210422042304240425042604270428042904300431043204330434043504360437043804390440044104420443044404450446044704480449045004510452045304540455045604570458045904600461046204630464046504660467046804690470047104720473047404750476047704780479048004810482048304840485048604870488048904900491049204930494049504960497049804990500' rand2[double precision]:4576
-COMMIT
-BEGIN
-table public.toasttable: UPDATE: id[integer]:1 toasted_col1[text]:'12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361136213631364136513661367136813691370137113721373137413751376137713781379138013811382138313841385138613871388138913901391139213931394139513961397139813991400140114021403140414051406140714081409141014111412141314141415141614171418141914201421142214231424142514261427142814291430143114321433143414351436143714381439144014411442144314441445144614471448144914501451145214531454145514561457145814591460146114621463146414651466146714681469147014711472147314741475147614771478147914801481148214831484148514861487148814891490149114921493149414951496149714981499150015011502150315041505150615071508150915101511151215131514151515161517151815191520152115221523152415251526152715281529153015311532153315341535153615371538153915401541154215431544154515461547154815491550155115521553155415551556155715581559156015611562156315641565156615671568156915701571157215731574157515761577157815791580158115821583158415851586158715881589159015911592159315941595159615971598159916001601160216031604160516061607160816091610161116121613161416151616161716181619162016211622162316241625162616271628162916301631163216331634163516361637163816391640164116421643164416451646164716481649165016511652165316541655165616571658165916601661166216631664166516661667166816691670167116721673167416751676167716781679168016811682168316841685168616871688168916901691169216931694169516961697169816991700170117021703170417051706170717081709171017111712171317141715171617171718171917201721172217231724172517261727172817291730173117321733173417351736173717381739174017411742174317441745174617471748174917501751175217531754175517561757175817591760176117621763176417651766176717681769177017711772177317741775177617771778177917801781178217831784178517861787178817891790179117921793179417951796179717981799180018011802180318041805180618071808180918101811181218131814181518161817181818191820182118221823182418251826182718281829183018311832183318341835183618371838183918401841184218431844184518461847184818491850185118521853185418551856185718581859186018611862186318641865186618671868186918701871187218731874187518761877187818791880188118821883188418851886188718881889189018911892189318941895189618971898189919001901190219031904190519061907190819091910191119121913191419151916191719181919192019211922192319241925192619271928192919301931193219331934193519361937193819391940194119421943194419451946194719481949195019511952195319541955195619571958195919601961196219631964196519661967196819691970197119721973197419751976197719781979198019811982198319841985198619871988198919901991199219931994199519961997199819992000' rand1[double precision]:79 toasted_col2[text]:null rand2[double precision]:1578
-COMMIT
-(143 rows)
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 \pset format aligned
 INSERT INTO toasttable(toasted_col1) SELECT string_agg(g.i::text, '') FROM generate_series(1, 2000) g(i);
 -- update of second column, first column unchanged
@@ -818,18 +576,10 @@
 DROP TABLE toasttable;
 \pset format unaligned
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-data
-BEGIN
-table public.toasttable: INSERT: id[integer]:3 toasted_col1[text]:'12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361136213631364136513661367136813691370137113721373137413751376137713781379138013811382138313841385138613871388138913901391139213931394139513961397139813991400140114021403140414051406140714081409141014111412141314141415141614171418141914201421142214231424142514261427142814291430143114321433143414351436143714381439144014411442144314441445144614471448144914501451145214531454145514561457145814591460146114621463146414651466146714681469147014711472147314741475147614771478147914801481148214831484148514861487148814891490149114921493149414951496149714981499150015011502150315041505150615071508150915101511151215131514151515161517151815191520152115221523152415251526152715281529153015311532153315341535153615371538153915401541154215431544154515461547154815491550155115521553155415551556155715581559156015611562156315641565156615671568156915701571157215731574157515761577157815791580158115821583158415851586158715881589159015911592159315941595159615971598159916001601160216031604160516061607160816091610161116121613161416151616161716181619162016211622162316241625162616271628162916301631163216331634163516361637163816391640164116421643164416451646164716481649165016511652165316541655165616571658165916601661166216631664166516661667166816691670167116721673167416751676167716781679168016811682168316841685168616871688168916901691169216931694169516961697169816991700170117021703170417051706170717081709171017111712171317141715171617171718171917201721172217231724172517261727172817291730173117321733173417351736173717381739174017411742174317441745174617471748174917501751175217531754175517561757175817591760176117621763176417651766176717681769177017711772177317741775177617771778177917801781178217831784178517861787178817891790179117921793179417951796179717981799180018011802180318041805180618071808180918101811181218131814181518161817181818191820182118221823182418251826182718281829183018311832183318341835183618371838183918401841184218431844184518461847184818491850185118521853185418551856185718581859186018611862186318641865186618671868186918701871187218731874187518761877187818791880188118821883188418851886188718881889189018911892189318941895189618971898189919001901190219031904190519061907190819091910191119121913191419151916191719181919192019211922192319241925192619271928192919301931193219331934193519361937193819391940194119421943194419451946194719481949195019511952195319541955195619571958195919601961196219631964196519661967196819691970197119721973197419751976197719781979198019811982198319841985198619871988198919901991199219931994199519961997199819992000' rand1[double precision]:6075 toasted_col2[text]:null rand2[double precision]:7574
-COMMIT
-BEGIN
-table public.toasttable: UPDATE: id[integer]:1 toasted_col1[text]:unchanged-toast-datum rand1[double precision]:79 toasted_col2[text]:'12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361136213631364136513661367136813691370137113721373137413751376137713781379138013811382138313841385138613871388138913901391139213931394139513961397139813991400140114021403140414051406140714081409141014111412141314141415141614171418141914201421142214231424142514261427142814291430143114321433143414351436143714381439144014411442144314441445144614471448144914501451145214531454145514561457145814591460146114621463146414651466146714681469147014711472147314741475147614771478147914801481148214831484148514861487148814891490149114921493149414951496149714981499150015011502150315041505150615071508150915101511151215131514151515161517151815191520152115221523152415251526152715281529153015311532153315341535153615371538153915401541154215431544154515461547154815491550155115521553155415551556155715581559156015611562156315641565156615671568156915701571157215731574157515761577157815791580158115821583158415851586158715881589159015911592159315941595159615971598159916001601160216031604160516061607160816091610161116121613161416151616161716181619162016211622162316241625162616271628162916301631163216331634163516361637163816391640164116421643164416451646164716481649165016511652165316541655165616571658165916601661166216631664166516661667166816691670167116721673167416751676167716781679168016811682168316841685168616871688168916901691169216931694169516961697169816991700170117021703170417051706170717081709171017111712171317141715171617171718171917201721172217231724172517261727172817291730173117321733173417351736173717381739174017411742174317441745174617471748174917501751175217531754175517561757175817591760176117621763176417651766176717681769177017711772177317741775177617771778177917801781178217831784178517861787178817891790179117921793179417951796179717981799180018011802180318041805180618071808180918101811181218131814181518161817181818191820182118221823182418251826182718281829183018311832183318341835183618371838183918401841184218431844184518461847184818491850185118521853185418551856185718581859186018611862186318641865186618671868186918701871187218731874187518761877187818791880188118821883188418851886188718881889189018911892189318941895189618971898189919001901190219031904190519061907190819091910191119121913191419151916191719181919192019211922192319241925192619271928192919301931193219331934193519361937193819391940194119421943194419451946194719481949195019511952195319541955195619571958195919601961196219631964196519661967196819691970197119721973197419751976197719781979198019811982198319841985198619871988198919901991199219931994199519961997199819992000' rand2[double precision]:1578
-COMMIT
-(6 rows)
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 -- done, free logical replication slot
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-data
-(0 rows)
+ERROR:  could not map filenumber "base/16384/16397" to relation OID
 \pset format aligned
 SELECT pg_drop_replication_slot('regression_slot');
  pg_drop_replication_slot 
diff -U3 /home/hayato/patchTest/postgres/contrib/test_decoding/expected/rewrite.out /home/hayato/patchTest/postgres/contrib/test_decoding/results/rewrite.out
--- /home/hayato/patchTest/postgres/contrib/test_decoding/expected/rewrite.out	2023-12-05 09:40:08.864621359 +0000
+++ /home/hayato/patchTest/postgres/contrib/test_decoding/results/rewrite.out	2024-05-20 04:49:00.802181926 +0000
@@ -116,21 +116,21 @@
 -- make old files go away
 CHECKPOINT;
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                                                                       data                                                                                       
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+                                                                                          data                                                                                          
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
  BEGIN
- table public.replication_example: INSERT: id[integer]:2 somedata[integer]:2 text[character varying]:null
+ table public.replication_example: INSERT: id[integer]:2 somedata[integer]:2 text[character varying]:null testcolumn1[integer]:null testcolumn2[integer]:null testcolumn3[integer]:null
  table public.replication_example: INSERT: id[integer]:3 somedata[integer]:3 text[character varying]:null testcolumn1[integer]:1
  COMMIT
  BEGIN
- table public.replication_example: INSERT: id[integer]:4 somedata[integer]:3 text[character varying]:null testcolumn1[integer]:null
+ table public.replication_example: INSERT: id[integer]:4 somedata[integer]:3 text[character varying]:null testcolumn1[integer]:null testcolumn2[integer]:null testcolumn3[integer]:null
  table public.replication_example: INSERT: id[integer]:5 somedata[integer]:4 text[character varying]:null testcolumn1[integer]:2 testcolumn2[integer]:1
  COMMIT
  BEGIN
- table public.replication_example: INSERT: id[integer]:6 somedata[integer]:5 text[character varying]:null testcolumn1[integer]:3 testcolumn2[integer]:null
+ table public.replication_example: INSERT: id[integer]:6 somedata[integer]:5 text[character varying]:null testcolumn1[integer]:3 testcolumn2[integer]:null testcolumn3[integer]:null
  COMMIT
  BEGIN
- table public.replication_example: INSERT: id[integer]:7 somedata[integer]:6 text[character varying]:null testcolumn1[integer]:4 testcolumn2[integer]:null
+ table public.replication_example: INSERT: id[integer]:7 somedata[integer]:6 text[character varying]:null testcolumn1[integer]:4 testcolumn2[integer]:null testcolumn3[integer]:null
  table public.replication_example: INSERT: id[integer]:8 somedata[integer]:7 text[character varying]:null testcolumn1[integer]:5 testcolumn2[integer]:null testcolumn3[integer]:1
  COMMIT
 (15 rows)
diff -U3 /home/hayato/patchTest/postgres/contrib/test_decoding/expected/toast.out /home/hayato/patchTest/postgres/contrib/test_decoding/results/toast.out
--- /home/hayato/patchTest/postgres/contrib/test_decoding/expected/toast.out	2024-01-23 12:00:13.548375422 +0000
+++ /home/hayato/patchTest/postgres/contrib/test_decoding/results/toast.out	2024-05-20 04:49:01.549181926 +0000
@@ -352,14 +352,7 @@
 DROP TABLE toasted_several;
 SELECT regexp_replace(data, '^(.{100}).*(.{100})$', '\1..\2') FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')
 WHERE data NOT LIKE '%INSERT: %';
-                                                                                               regexp_replace                                                                                               
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
- BEGIN
- table public.toasted_several: UPDATE: old-key: id[integer]:1 toasted_key[text]:'98765432109876543210..7654321098765432109876543210987654321098765432109876543210' toasted_col2[text]:unchanged-toast-datum
- table public.toasted_several: DELETE: id[integer]:1 toasted_key[text]:'98765432109876543210987654321..876543210987654321098765432109876543210987654321098765432109876543210987654321098765432109876543210'
- COMMIT
-(4 rows)
-
+ERROR:  could not map filenumber "base/16384/16709" to relation OID
 /*
  * Test decoding relation rewrite with toast. The insert into tbl2 within the
  * same transaction is there to check that there is no remaining toast_hash not
@@ -374,14 +367,7 @@
 INSERT INTO tbl2 VALUES(1);
 commit;
 SELECT substr(data, 1, 200) FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                                                                                  substr                                                                                                  
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
- BEGIN
- table public.tbl1: INSERT: a[integer]:1 b[text]:'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa
- table public.tbl2: INSERT: a[integer]:1
- COMMIT
-(4 rows)
-
+ERROR:  could not map filenumber "base/16384/16709" to relation OID
 SELECT pg_drop_replication_slot('regression_slot');
  pg_drop_replication_slot 
 --------------------------
diff -U3 /home/hayato/patchTest/postgres/contrib/test_decoding/expected/decoding_into_rel.out /home/hayato/patchTest/postgres/contrib/test_decoding/results/decoding_into_rel.out
--- /home/hayato/patchTest/postgres/contrib/test_decoding/expected/decoding_into_rel.out	2023-10-18 07:11:15.274329904 +0000
+++ /home/hayato/patchTest/postgres/contrib/test_decoding/results/decoding_into_rel.out	2024-05-20 04:49:01.713181926 +0000
@@ -66,43 +66,11 @@
     SELECT data FROM pg_logical_slot_peek_changes(slot_name, NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
 END$$ LANGUAGE plpgsql;
 SELECT * FROM slot_changes_wrapper('regression_slot');
-                                                                                          slot_changes_wrapper                                                                                          
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
- BEGIN
- table public.changeresult: INSERT: data[text]:'BEGIN'
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''BEGIN'''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''table public.somechange: INSERT: id[integer]:1'''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''COMMIT'''
- table public.changeresult: INSERT: data[text]:'COMMIT'
- table public.changeresult: INSERT: data[text]:'BEGIN'
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''BEGIN'''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''table public.changeresult: INSERT: data[text]:''''BEGIN'''''''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''table public.changeresult: INSERT: data[text]:''''table public.somechange: INSERT: id[integer]:1'''''''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''table public.changeresult: INSERT: data[text]:''''COMMIT'''''''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''COMMIT'''
- table public.changeresult: INSERT: data[text]:'COMMIT'
- COMMIT
-(14 rows)
-
+ERROR:  could not map filenumber "base/16384/16767" to relation OID
+CONTEXT:  SQL statement "SELECT data FROM pg_logical_slot_peek_changes(slot_name, NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1')"
+PL/pgSQL function slot_changes_wrapper(name) line 3 at RETURN QUERY
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                                                                                  data                                                                                                  
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
- BEGIN
- table public.changeresult: INSERT: data[text]:'BEGIN'
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''BEGIN'''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''table public.somechange: INSERT: id[integer]:1'''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''COMMIT'''
- table public.changeresult: INSERT: data[text]:'COMMIT'
- table public.changeresult: INSERT: data[text]:'BEGIN'
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''BEGIN'''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''table public.changeresult: INSERT: data[text]:''''BEGIN'''''''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''table public.changeresult: INSERT: data[text]:''''table public.somechange: INSERT: id[integer]:1'''''''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''table public.changeresult: INSERT: data[text]:''''COMMIT'''''''
- table public.changeresult: INSERT: data[text]:'table public.changeresult: INSERT: data[text]:''COMMIT'''
- table public.changeresult: INSERT: data[text]:'COMMIT'
- COMMIT
-(14 rows)
-
+ERROR:  could not map filenumber "base/16384/16767" to relation OID
 SELECT 'stop' FROM pg_drop_replication_slot('regression_slot');
  ?column? 
 ----------
diff -U3 /home/hayato/patchTest/postgres/contrib/test_decoding/expected/prepared.out /home/hayato/patchTest/postgres/contrib/test_decoding/results/prepared.out
--- /home/hayato/patchTest/postgres/contrib/test_decoding/expected/prepared.out	2023-10-18 07:11:15.275329904 +0000
+++ /home/hayato/patchTest/postgres/contrib/test_decoding/results/prepared.out	2024-05-20 04:49:01.780181926 +0000
@@ -40,32 +40,7 @@
 DROP TABLE test_prepared2;
 -- show results
 SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
-                                  data                                   
--------------------------------------------------------------------------
- BEGIN
- table public.test_prepared1: INSERT: id[integer]:1
- COMMIT
- BEGIN
- table public.test_prepared1: INSERT: id[integer]:2
- COMMIT
- BEGIN
- table public.test_prepared1: INSERT: id[integer]:4
- COMMIT
- BEGIN
- table public.test_prepared2: INSERT: id[integer]:7
- COMMIT
- BEGIN
- table public.test_prepared1: INSERT: id[integer]:5
- table public.test_prepared1: INSERT: id[integer]:6 data[text]:'frakbar'
- COMMIT
- BEGIN
- table public.test_prepared1: INSERT: id[integer]:8 data[text]:null
- COMMIT
- BEGIN
- table public.test_prepared2: INSERT: id[integer]:9
- COMMIT
-(22 rows)
-
+ERROR:  could not map filenumber "base/16384/16773" to relation OID
 SELECT pg_drop_replication_slot('regression_slot');
  pg_drop_replication_slot 
 --------------------------
diff -U3 /home/hayato/patchTest/postgres/contrib/test_decoding/expected/stream.out /home/hayato/patchTest/postgres/contrib/test_decoding/results/stream.out
--- /home/hayato/patchTest/postgres/contrib/test_decoding/expected/stream.out	2024-04-23 00:42:15.088956778 +0000
+++ /home/hayato/patchTest/postgres/contrib/test_decoding/results/stream.out	2024-05-20 04:49:10.264181926 +0000
@@ -31,6 +31,37 @@
 ----------------------------------------------------------
  opening a streamed block for transaction
  streaming message: transactional: 1 prefix: test, sz: 50
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
  closing a streamed block for transaction
  aborting streamed (sub)transaction
  opening a streamed block for transaction
@@ -56,7 +87,7 @@
  streaming change for transaction
  closing a streamed block for transaction
  committing streamed transaction
-(27 rows)
+(58 rows)
 
 -- streaming test for toast changes
 ALTER TABLE stream_test ALTER COLUMN data set storage external;
diff -U3 /home/hayato/patchTest/postgres/contrib/test_decoding/expected/twophase_stream.out /home/hayato/patchTest/postgres/contrib/test_decoding/results/twophase_stream.out
--- /home/hayato/patchTest/postgres/contrib/test_decoding/expected/twophase_stream.out	2024-04-23 00:42:15.088956778 +0000
+++ /home/hayato/patchTest/postgres/contrib/test_decoding/results/twophase_stream.out	2024-05-20 04:49:10.619181926 +0000
@@ -33,6 +33,37 @@
 ----------------------------------------------------------
  opening a streamed block for transaction
  streaming message: transactional: 1 prefix: test, sz: 50
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
  closing a streamed block for transaction
  aborting streamed (sub)transaction
  opening a streamed block for transaction
@@ -58,7 +89,7 @@
  streaming change for transaction
  closing a streamed block for transaction
  preparing streamed transaction 'test1'
-(27 rows)
+(58 rows)
 
 COMMIT PREPARED 'test1';
 --should show the COMMIT PREPARED and the other changes in the transaction
@@ -89,9 +120,40 @@
 ----------------------------------------------------------
  opening a streamed block for transaction
  streaming message: transactional: 1 prefix: test, sz: 50
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
+ streaming change for transaction
  closing a streamed block for transaction
  aborting streamed (sub)transaction
-(4 rows)
+(35 rows)
 
 COMMIT PREPARED 'test1_nodecode';
 -- should show the inserts but not show a COMMIT PREPARED but a COMMIT
diff -U3 /home/hayato/patchTest/postgres/contrib/test_decoding/expected/filter.out /home/hayato/patchTest/postgres/contrib/test_decoding/results/filter.out
--- /home/hayato/patchTest/postgres/contrib/test_decoding/expected/filter.out	2024-05-20 03:13:35.881181926 +0000
+++ /home/hayato/patchTest/postgres/contrib/test_decoding/results/filter.out	2024-05-20 04:49:10.648181926 +0000
@@ -0,0 +1,29 @@
+-- predictability
+SET synchronous_commit = on;
+-- define table which will be filtered
+CREATE TABLE test_tobeskipped (a int);
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column? 
+----------
+ init
+(1 row)
+
+INSERT INTO test_tobeskipped VALUES (generate_series(1, 10));
+-- check that changes to test_tobeskipped are not output
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
+                         data                         
+------------------------------------------------------
+ BEGIN
+ table public.test_tobeskipped: INSERT: a[integer]:1
+ table public.test_tobeskipped: INSERT: a[integer]:2
+ table public.test_tobeskipped: INSERT: a[integer]:3
+ table public.test_tobeskipped: INSERT: a[integer]:4
+ table public.test_tobeskipped: INSERT: a[integer]:5
+ table public.test_tobeskipped: INSERT: a[integer]:6
+ table public.test_tobeskipped: INSERT: a[integer]:7
+ table public.test_tobeskipped: INSERT: a[integer]:8
+ table public.test_tobeskipped: INSERT: a[integer]:9
+ table public.test_tobeskipped: INSERT: a[integer]:10
+ COMMIT
+(12 rows)
+
#8Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#6)
1 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Tue, Mar 19, 2024 at 1:49 PM Ajin Cherian <itsajin@gmail.com> wrote:

Of course you can, but this will only convert disk space into memory
space.
For details, please see the case in Email [1].

[1]
/messages/by-id/CAGfChW51P944nM5h0HTV9HistvVfwBxNaMt_s-OZ9t=uXz+Zbg@mail.gmail.com

Regards, lijie

In some testing, I see a crash:
(gdb) bt
#0 0x00007fa5bcbfd277 in raise () from /lib64/libc.so.6
#1 0x00007fa5bcbfe968 in abort () from /lib64/libc.so.6
#2 0x00000000009e0940 in ExceptionalCondition (
conditionName=conditionName@entry=0x7fa5ab8b9842 "RelationSyncCache !=
NULL",
fileName=fileName@entry=0x7fa5ab8b9820 "pgoutput.c",
lineNumber=lineNumber@entry=1991)
at assert.c:66
#3 0x00007fa5ab8b7804 in get_rel_sync_entry (data=data@entry=0x2492288,
relation=relation@entry=0x7fa5be30a768) at pgoutput.c:1991
#4 0x00007fa5ab8b7cda in pgoutput_table_filter (ctx=<optimized out>,
relation=0x7fa5be30a768,
change=0x24c5c20) at pgoutput.c:1671
#5 0x0000000000813761 in filter_by_table_cb_wrapper (ctx=ctx@entry=0x2491fd0,

relation=relation@entry=0x7fa5be30a768, change=change@entry=0x24c5c20)
at logical.c:1268
#6 0x000000000080e20f in FilterByTable (ctx=ctx@entry=0x2491fd0,
change=change@entry=0x24c5c20)
at decode.c:690
#7 0x000000000080e8e3 in DecodeInsert (ctx=ctx@entry=0x2491fd0,
buf=buf@entry=0x7fff0db92550)
at decode.c:1070
#8 0x000000000080f43d in heap_decode (ctx=ctx@entry=0x2491fd0,
buf=buf@entry=0x7fff0db92550)
at decode.c:485
#9 0x000000000080eca6 in LogicalDecodingProcessRecord
(ctx=ctx@entry=0x2491fd0,
record=0x2492368)
at decode.c:118
#10 0x000000000081338f in DecodingContextFindStartpoint
(ctx=ctx@entry=0x2491fd0)
at logical.c:672
#11 0x000000000083c650 in CreateReplicationSlot (cmd=cmd@entry=0x2490970)
at walsender.c:1323
#12 0x000000000083fd48 in exec_replication_command (
cmd_string=cmd_string@entry=0x239c880 "CREATE_REPLICATION_SLOT
\"pg_16387_sync_16384_7371301304766135621\" LOGICAL pgoutput (SNAPSHOT
'use')") at walsender.c:2116

The reason for the crash is that the RelationSyncCache was NULL prior to
reaching a consistent point.
Hi li jie, I see that you created a new thread with an updated version of
this patch [1]/messages/by-id/CAGfChW7+ZMN4_NHPgz24MM42HVO83ecr9TLfpihJ=M0s1GkXFw@mail.gmail.com. I used that patch and addressed the crash seen above,
rebased the patch and addressed a few other comments.
I'm happy to help you with this patch and address comments if you are not
available.

regards,
Ajin Cherian
Fujitsu Australia
[1]: /messages/by-id/CAGfChW7+ZMN4_NHPgz24MM42HVO83ecr9TLfpihJ=M0s1GkXFw@mail.gmail.com
/messages/by-id/CAGfChW7+ZMN4_NHPgz24MM42HVO83ecr9TLfpihJ=M0s1GkXFw@mail.gmail.com

Attachments:

v2-0001-Reduce-useless-changes-before-reassemble-during-l.patchapplication/octet-stream; name=v2-0001-Reduce-useless-changes-before-reassemble-during-l.patchDownload
From e5484c1e46457103f8e3d3a5e126ae4825c80fc6 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <ajinc@fast.au.fujitsu.com>
Date: Tue, 21 May 2024 23:57:55 -0400
Subject: [PATCH v2] Reduce useless changes before reassemble during logical
 replication.

In order to reduce unnecessary logical replication, irrelevant relationship
changes can be filtered out before reorganizing transaction fragments.
This can effectively reduce useless changes and prevent storage space from
being filled up with irrelevant data.

By design, Added a callback LogicalDecodeFilterByRelCB for the output plugin.
We implemented a function pgoutput_table_filter for pgoutput. And RelationSyncCache
is reused to determine whether a relationship-related change should be filtered out.
referring to the implementation of the function pgoutput_change, currently only
insert/update/delete is can filtered, and other types of changes are not considered.
Perhaps more detailed analysis can be done and more filters can be filtered.

Most of the code in the FilterByTable function is transplanted from the ReorderBufferProcessTXN
function, which can be called before the ReorderBufferQueueChange function.It is
also the encapsulation of the callback function filter_by_table_cb in logical.c.

In general, this patch concentrates the filtering of changes in the ReorderBufferProcessTXN
function and the pgoutput_change function into the FilterByTable function, and calls
it before the ReorderBufferQueueChange function.
Author: Lijie
---
 src/backend/replication/logical/decode.c    | 157 +++++++++++++++++++++++++++-
 src/backend/replication/logical/logical.c   |  31 ++++++
 src/backend/replication/pgoutput/pgoutput.c |  89 ++++++++++------
 src/include/replication/logical.h           |   2 +
 src/include/replication/output_plugin.h     |   7 ++
 src/test/subscription/t/034_table_filter.pl |  91 ++++++++++++++++
 6 files changed, 339 insertions(+), 38 deletions(-)
 create mode 100644 src/test/subscription/t/034_table_filter.pl

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 7a86f84..01b43cc 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -33,12 +33,14 @@
 #include "access/xlogreader.h"
 #include "access/xlogrecord.h"
 #include "catalog/pg_control.h"
+#include "common/string.h"
 #include "replication/decode.h"
 #include "replication/logical.h"
 #include "replication/message.h"
 #include "replication/reorderbuffer.h"
 #include "replication/snapbuild.h"
 #include "storage/standbydefs.h"
+#include "utils/relfilenumbermap.h"
 
 /* individual record(group)'s handlers */
 static void DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf);
@@ -152,7 +154,7 @@ xlog_decode(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 		case XLOG_PARAMETER_CHANGE:
 			{
 				xl_parameter_change *xlrec =
-					(xl_parameter_change *) XLogRecGetData(buf->record);
+				(xl_parameter_change *) XLogRecGetData(buf->record);
 
 				/*
 				 * If wal_level on the primary is reduced to less than
@@ -578,6 +580,138 @@ FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return filter_by_origin_cb_wrapper(ctx, origin_id);
 }
 
+static bool
+FilterByTable(LogicalDecodingContext *ctx, ReorderBufferChange *change)
+{
+	ReorderBuffer *rb = ctx->reorder;
+	Relation	relation = NULL;
+	SnapBuild  *builder = ctx->snapshot_builder;
+	Oid			reloid;
+	bool		result = false;
+	bool		using_subtxn;
+
+	if (ctx->callbacks.filter_by_table_cb == NULL)
+		return false;
+
+	if(SnapBuildCurrentState(builder) < SNAPBUILD_CONSISTENT)
+		return false;
+
+	switch (change->action)
+	{
+			/* intentionally fall through */
+		case REORDER_BUFFER_CHANGE_INSERT:
+		case REORDER_BUFFER_CHANGE_UPDATE:
+		case REORDER_BUFFER_CHANGE_DELETE:
+			break;
+		default:
+			return false;
+	}
+
+	/*
+	 * Decoding needs access to syscaches et al., which in turn use
+	 * heavyweight locks and such. Thus we need to have enough state around to
+	 * keep track of those.  The easiest way is to simply use a transaction
+	 * internally.  That also allows us to easily enforce that nothing writes
+	 * to the database by checking for xid assignments.
+	 *
+	 * When we're called via the SQL SRF there's already a transaction
+	 * started, so start an explicit subtransaction there.
+	 */
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by table");
+	else
+		StartTransactionCommand();
+
+	reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
+								  change->data.tp.rlocator.relNumber);
+	if (reloid == InvalidOid)
+	{
+		result = true;
+		goto filter_done;
+	}
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid,
+			 relpathperm(change->data.tp.rlocator,
+						 MAIN_FORKNUM));
+
+	if (!RelationIsLogicallyLogged(relation))
+	{
+		result = true;
+		goto filter_done;
+	}
+
+	/*
+	 * Ignore temporary heaps created during DDL unless the plugin has asked
+	 * for them.
+	 */
+	if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		result = true;
+		goto filter_done;
+	}
+
+	/*
+	 * For now ignore sequence changes entirely. Most of the time they don't
+	 * log changes using records we understand, so it doesn't make sense to
+	 * handle the few cases we do.
+	 */
+	if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		result = true;
+		goto filter_done;
+	}
+
+	if (IsToastRelation(relation))
+	{
+		Oid			real_reloid = InvalidOid;
+
+		/* pg_toast_ len is 9 */
+		char	   *toast_name = RelationGetRelationName(relation);
+		char	   *start_ch = &toast_name[9];
+
+		real_reloid = strtoint(start_ch, NULL, 10);
+
+		if (real_reloid == InvalidOid)
+			elog(ERROR, "cannot get the real table oid for toast table %s, error: %m", toast_name);
+
+		RelationClose(relation);
+
+		relation = RelationIdGetRelation(real_reloid);
+
+		if (!RelationIsValid(relation))
+			elog(ERROR, "could not open real relation with OID %u (for toast table filenumber \"%s\")",
+				 reloid,
+				 relpathperm(change->data.tp.rlocator,
+							 MAIN_FORKNUM));
+	}
+
+	result = filter_by_table_cb_wrapper(ctx, relation, change);
+
+filter_done:
+
+	if (result && RelationIsValid(relation))
+		elog(DEBUG1, "logical filter change by table %s", RelationGetRelationName(relation));
+
+	if (RelationIsValid(relation))
+		RelationClose(relation);
+
+	AbortCurrentTransaction();
+
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	if (result)
+		ReorderBufferReturnChange(rb, change, false);
+
+	return result;
+}
+
 /*
  * Handle rmgr LOGICALMSG_ID records for LogicalDecodingProcessRecord().
  */
@@ -937,6 +1071,9 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 	change->data.tp.clear_toast_afterwards = true;
 
+	if (FilterByTable(ctx, change))
+		return;
+
 	ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r), buf->origptr,
 							 change,
 							 xlrec->flags & XLH_INSERT_ON_TOAST_RELATION);
@@ -1006,6 +1143,9 @@ DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 	change->data.tp.clear_toast_afterwards = true;
 
+	if (FilterByTable(ctx, change))
+		return;
+
 	ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r), buf->origptr,
 							 change, false);
 }
@@ -1062,6 +1202,9 @@ DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 	change->data.tp.clear_toast_afterwards = true;
 
+	if (FilterByTable(ctx, change))
+		return;
+
 	ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r), buf->origptr,
 							 change, false);
 }
@@ -1198,11 +1341,14 @@ DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 		else
 			change->data.tp.clear_toast_afterwards = false;
 
-		ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r),
-								 buf->origptr, change, false);
-
 		/* move to the next xl_multi_insert_tuple entry */
 		data += datalen;
+
+		if (FilterByTable(ctx, change))
+			continue;
+
+		ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r),
+								 buf->origptr, change, false);
 	}
 	Assert(data == tupledata + tuplelen);
 }
@@ -1237,6 +1383,9 @@ DecodeSpecConfirm(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 
 	change->data.tp.clear_toast_afterwards = true;
 
+	if (FilterByTable(ctx, change))
+		return;
+
 	ReorderBufferQueueChange(ctx->reorder, XLogRecGetXid(r), buf->origptr,
 							 change, false);
 }
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 97a4d99..49ac184 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -1242,6 +1242,37 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+bool
+filter_by_table_cb_wrapper(LogicalDecodingContext *ctx, Relation relation, ReorderBufferChange *change)
+{
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_by_table";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_by_table_cb(ctx, relation, change);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index d2b35cf..3d1de74 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -56,6 +56,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_table_filter(struct LogicalDecodingContext *ctx,
+								  Relation relation,
+								  ReorderBufferChange *change);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -258,6 +261,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_by_table_cb = pgoutput_table_filter;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1414,9 +1418,6 @@ pgoutput_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
 	TupleTableSlot *old_slot = NULL;
 	TupleTableSlot *new_slot = NULL;
 
-	if (!is_publishable_relation(relation))
-		return;
-
 	/*
 	 * Remember the xid for the change in streaming mode. We need to send xid
 	 * with each change in the streaming mode so that subscriber can make
@@ -1427,37 +1428,6 @@ pgoutput_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
 		xid = change->txn->xid;
 
 	relentry = get_rel_sync_entry(data, relation);
-
-	/* First check the table filter */
-	switch (action)
-	{
-		case REORDER_BUFFER_CHANGE_INSERT:
-			if (!relentry->pubactions.pubinsert)
-				return;
-			break;
-		case REORDER_BUFFER_CHANGE_UPDATE:
-			if (!relentry->pubactions.pubupdate)
-				return;
-			break;
-		case REORDER_BUFFER_CHANGE_DELETE:
-			if (!relentry->pubactions.pubdelete)
-				return;
-
-			/*
-			 * This is only possible if deletes are allowed even when replica
-			 * identity is not defined for a table. Since the DELETE action
-			 * can't be published, we simply return.
-			 */
-			if (!change->data.tp.oldtuple)
-			{
-				elog(DEBUG1, "didn't send DELETE change because of missing oldtuple");
-				return;
-			}
-			break;
-		default:
-			Assert(false);
-	}
-
 	/* Avoid leaking memory by using and resetting our own context */
 	old = MemoryContextSwitchTo(data->context);
 
@@ -1684,6 +1654,57 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 }
 
 /*
+ * Return true if the relation has not been published, false otherwise.
+ */
+static bool
+pgoutput_table_filter(struct LogicalDecodingContext *ctx,
+					  Relation relation,
+					  ReorderBufferChange *change)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *relentry;
+	ReorderBufferChangeType action = change->action;
+
+	if (!is_publishable_relation(relation))
+		return true;
+
+	relentry = get_rel_sync_entry(data, relation);
+
+	/* First check the table filter */
+	switch (action)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			if (!relentry->pubactions.pubinsert)
+				return true;
+			break;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			if (!relentry->pubactions.pubupdate)
+				return true;
+			break;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			if (!relentry->pubactions.pubdelete)
+				return true;
+
+			/*
+			 * This is only possible if deletes are allowed even when replica
+			 * identity is not defined for a table. Since the DELETE action
+			 * can't be published, we simply return.
+			 */
+			if (!change->data.tp.oldtuple)
+			{
+				elog(DEBUG1, "didn't send DELETE change because of missing oldtuple");
+				return true;
+			}
+			break;
+		default:
+			Assert(false);
+	}
+
+	return false;
+}
+
+
+/*
  * Shutdown the output plugin.
  *
  * Note, we don't need to clean the data->context and data->cachectx as
diff --git a/src/include/replication/logical.h b/src/include/replication/logical.h
index aff38e8..dae1411 100644
--- a/src/include/replication/logical.h
+++ b/src/include/replication/logical.h
@@ -145,6 +145,8 @@ extern void LogicalConfirmReceivedLocation(XLogRecPtr lsn);
 extern bool filter_prepare_cb_wrapper(LogicalDecodingContext *ctx,
 									  TransactionId xid, const char *gid);
 extern bool filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id);
+extern bool filter_by_table_cb_wrapper(LogicalDecodingContext *ctx, Relation relation,
+										ReorderBufferChange *change);
 extern void ResetLogicalStreamingState(void);
 extern void UpdateDecodingStats(LogicalDecodingContext *ctx);
 
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 44988eb..030eb5a 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -97,6 +97,12 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
 											   RepOriginId origin_id);
 
 /*
+ * Filter changes by table.
+ */
+typedef bool (*LogicalDecodeFilterByRelCB) (struct LogicalDecodingContext *ctx,
+											Relation relation, ReorderBufferChange *change);
+
+/*
  * Called to shutdown an output plugin.
  */
 typedef void (*LogicalDecodeShutdownCB) (struct LogicalDecodingContext *ctx);
@@ -222,6 +228,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterByRelCB filter_by_table_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/test/subscription/t/034_table_filter.pl b/src/test/subscription/t/034_table_filter.pl
new file mode 100644
index 0000000..8d4f962
--- /dev/null
+++ b/src/test/subscription/t/034_table_filter.pl
@@ -0,0 +1,91 @@
+# Copyright (c) 2021-2023, PostgreSQL Global Development Group
+
+# Basic logical replication test
+use strict;
+use warnings;
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+# Initialize publisher node
+my $node_publisher = PostgreSQL::Test::Cluster->new('publisher');
+$node_publisher->init(allows_streaming => 'logical');
+$node_publisher->append_conf('postgresql.conf', 'logical_decoding_work_mem = 64kB');
+$node_publisher->start;
+
+# Create subscriber node
+my $node_subscriber = PostgreSQL::Test::Cluster->new('subscriber');
+$node_subscriber->init;
+$node_subscriber->start;
+
+
+# Create some preexisting content on publisher
+$node_publisher->safe_psql('postgres',
+	"create table tbl_pub(id int, val1 text, val2 text,size int);");
+$node_publisher->safe_psql('postgres',
+	"create table tbl_t1(id int, val1 text, val2 text,size int);");
+$node_publisher->safe_psql('postgres',
+	"CREATE PUBLICATION mypub FOR TABLE public.tbl_pub;");
+$node_publisher->safe_psql('postgres',
+qq(
+CREATE OR REPLACE FUNCTION check_replication_status() RETURNS VOID AS \$\$
+DECLARE
+    replication_record pg_stat_replication;
+BEGIN
+    LOOP
+        SELECT *
+        INTO replication_record
+        FROM pg_stat_replication
+        WHERE application_name = 'mysub';
+        
+        IF replication_record.replay_lsn = replication_record.write_lsn THEN
+            EXIT;
+        END IF;
+    
+        PERFORM pg_sleep(1);
+    END LOOP;
+END;
+\$\$ LANGUAGE plpgsql;));
+
+# Create some preexisting content on subscriber
+my $publisher_connstr = $node_publisher->connstr . ' dbname=postgres';
+$node_subscriber->safe_psql('postgres', 
+    "create table tbl_pub(id int, val1 text, val2 text,size text);");
+$node_subscriber->safe_psql('postgres',
+    "create table tbl_t1(id int, val1 text, val2 text,size text);");
+$node_subscriber->safe_psql('postgres',
+	"CREATE SUBSCRIPTION mysub CONNECTION '$publisher_connstr' PUBLICATION mypub"
+);
+
+# Wait for initial table sync to finish
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'mysub');
+
+# test filter
+$node_publisher->safe_psql('postgres',
+qq(BEGIN;
+insert into tbl_t1 select 1, 'xyzzy', 'abcba', sum(size) from pg_ls_replslotdir('mysub');
+insert into tbl_t1 select i,repeat('xyzzy', i),repeat('abcba',i),(select sum(size) from pg_ls_replslotdir('mysub')) from generate_series(2,9999) i;
+update tbl_t1 set val2 = repeat('xyzzy',id) where id > 1 and id < 10001;
+select check_replication_status();
+insert into tbl_t1 select 10001, 'xyzzy', 'abcba', sum(size) from pg_ls_replslotdir('mysub');
+COMMIT;)
+);
+
+my $minsize =   $node_publisher->safe_psql('postgres',
+	"select size from tbl_t1 order by size asc limit 1;");
+my $maxsize =   $node_publisher->safe_psql('postgres',
+	"select size from tbl_t1 order by size desc limit 1;");
+is($minsize, $maxsize, 'check decode filter table between maxsize and minsize');
+
+
+my $fristrow =   $node_publisher->safe_psql('postgres',
+	"select size from tbl_t1 where id = 1;");
+my $lastrow =   $node_publisher->safe_psql('postgres',
+	"select size from tbl_t1 where id = 10001;");
+is($minsize, $maxsize, 'check decode filter table between fristrow and lastrow');
+
+is($minsize, $lastrow, 'check decode filter table between minsize and lastrow');
+
+print "minsize: " . $minsize . "maxsize: " . $maxsize ."fristrow: " . $fristrow ."lastrow: " . $lastrow . "\n";
+
+done_testing();
\ No newline at end of file
-- 
1.8.3.1

#9Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#8)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Wed, May 22, 2024 at 2:17 PM Ajin Cherian <itsajin@gmail.com> wrote:

The reason for the crash is that the RelationSyncCache was NULL prior to
reaching a consistent point.
Hi li jie, I see that you created a new thread with an updated version of
this patch [1]. I used that patch and addressed the crash seen above,
rebased the patch and addressed a few other comments.
I'm happy to help you with this patch and address comments if you are not
available.

regards,
Ajin Cherian
Fujitsu Australia
[1] -
/messages/by-id/CAGfChW7+ZMN4_NHPgz24MM42HVO83ecr9TLfpihJ=M0s1GkXFw@mail.gmail.com

I was discussing this with Kuroda-san who made a patch to add a table
filter with test_decoding plugin. The filter does nothing, just returns
false( doesn't filter anything) and I see that 8 test_decoding tests fail.
In my analysis, I could see that some of the failures were because the new
filter logic was accessing the relation cache using the latest snapshot for
relids which was getting incorrect relation information while decoding
attribute values.
for eg:
CREATE TABLE replication_example(id SERIAL PRIMARY KEY, somedata int, text
varchar(120));
BEGIN;
INSERT INTO replication_example(somedata, text) VALUES (1, 1);
INSERT INTO replication_example(somedata, text) VALUES (1, 2);
COMMIT;
ALTER TABLE replication_example RENAME COLUMN text TO somenum;
SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL,
'include-xids', '0', 'skip-empty-xacts', '1');

Here, after decoding, the changes for the INSERT, were reflecting the new
column name (somenum) which was altered in a later transaction. This is
because the new Filterby Table() logic was getting relcache with latest
snapshot and does not use a historic snapshot like logical decoding should
be doing. This is because the changes are at the decode.c level and not the
reorderbuffer level and does not have access to the txn from the
reorderbuffer. This problem could be fixed by invalidating the cache in the
FilterByTable() logic, but this does not solve other problems like the
table name itself is changed in a later transaction. I think the patch has
a fundamental problem that the table filter logic does not respect the
snapshot of the transaction being decoded. The historic snapshot is
currently only set up when the actual changes are committed or streamed at
ReorderBufferProcessTXN().

If the purpose of the patch is to filter out unnecessary changes prior to
actual decode, then it will use an invalid snapshot and have lots of
problems. Otherwise this logic has to be moved to the reorderbuffer level
and there will be a big overhead of extracting reorderbuffer while each
change is queued in memory/disk.
regards,
Ajin Cherian
Fujitsu Australia

#10Zhijie Hou (Fujitsu)
houzj.fnst@fujitsu.com
In reply to: Ajin Cherian (#9)
RE: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi,

I think the proposed feature is interesting, and after reviewing the
original discussion thread, I'd like to help move it forward. To that end, I've
summarized the key points of the discussion so far, including the challenges
highlighted in the thread and some issues I identified off-list, along with
potential solutions. My colleague will share a new POC patch set that has been
developed based on these lines.
(Note that the previous email was blocked, this is the second attempt)

The original proposal in this thread suggests filtering out changes that won't
be published before decoding. This approach could prevent the serialization of
numerous unnecessary records to disk when using non-streaming mode for large
transactions. I find this to be a valid proposal. Although a recent commit has
enabled streaming mode for built-in logical replication by default, this
strategy could also save memory and CPU cycles in streaming mode, especially
when most changes can be filtered out.

While the proposal is interesting, several issues have been identified both in
this thread and off-list:

- Performance

Determining whether to filter a change requires information about the relation
or publication from the catalog, requiring a transaction to be started. When
most changes in a transaction are unfilterable, the overhead of starting a
transaction for each record is significant.

This is not a big issue because we can cache whether a change corresponding to
a table is filterable using a hash table.
However, a single hash search may not be sufficiently cheap if performed for
every record before decoding. In my tests, this caching approach produced
noticeable overhead (around 4%), mainly due to the cost of computing the hash
key and function calls in a hot path

--4.86%--hash_search
|--3.55%--tag_hash
| --2.96%--hash_bytes
--0.37%--hash_search_with_hash_value

Such overhead may not be acceptable for all users, given valid use cases like
publishing most tables on an instance (e.g., during an upgrade using logical
replication), where this filtering wouldn't provide a benefit and would only
incur overhead.

A tiny approach to minimize overhead is to suspend filtering for a certain
period when an unfilterable change is encountered. In other words, continue
filtering changes (using hash cache) if the last record was successfully
filtered out. If an unfilterable change is found, skip filtering the next 100
(an arbitrary number) changes. This approach aims to reduce frequent hash
searches when most changes can't be filtered out, lowering the overhead to less
than 1% on my machine. (This is a simple idea, and better algorithms could
exist.)

- Snapshot construction

The main challenge discussed in this thread is constructing a correct historic
snapshot for filtering purposes. Creating a historic snapshot while decoding an
in-progress transaction is feasible, as it's already done in streaming mode.
However, there could be pending records (INTERNAL_SNAPSHOT, COMMAND_ID, or
INVALIDATION) that will update the built-in snapshot. We shouldn't use a
current snapshot without processing these records.

But thinking from another perspective, why not perform filtering only when
there are no such pending records. By tracking whether the decoder has any of
these records, we could skip filtering if they are present. These special
records are generally generated only during DDL execution, which shouldn't be
frequent; thus, the filter could still benefit many scenarios.

- new output plugin callback

To perform filtering, a new callback (e.g., change_filter) would be needed to
invoke before decoding the record. Using pgoutput as an example, we would call
get_rel_sync_entry() in the callback to determine if a change corresponding to
a table can be filtered.

get_rel_sync_entry() requires catalog access if it doesn't find the cache entry
in the hash table. But as mentioned in the "Performance" section, we need to
minimize catalog access and transaction start/stop. So, ideally, the callback
should return whether it needs catalog access, allowing the caller
(reorderbuffer) to start a transaction if necessary.

The callback interface could be:

(Note: The function or parameter names can be refined; the example just
provided what's the input and output information.)

typedef bool (LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext ctx,
Oid relid,
ReorderBufferChangeType type,
bool in_txn, bool *cache_valid);

To signal that decoding should be skipped for the given change type, it returns
true; false otherwise. The in_txn parameter indicates whether the callback is
invoked within a transaction block. When in_txn is false, and if making a
decision to filter a change requires being inside a transaction block, such as
needing access to the catalog, set *cache_valid to false. The caller should
invoke this for each record, and invoke this without starting a transaction, if
the returned cache_valid is false, then the caller should should reinvoke the
callback after starting a transaction.

Another alternative approach in Li jie patch[1]/messages/by-id/CAGfChW7XpZGkxOpHZnv69+H_AyKzbffcrumyNu4Fz0u+1ADPxA@mail.gmail.com is having the output plugin
callback always provide all record types that can be published for a relation
and cache them at the reorderbuffer level. However, I think this isn't feasible
because the reorderbuffer isn't (and needn't to be) aware of when to invalidate
such a cache. Although we could register the cache invalidation callback like
pgoutput does in init_rel_sync_cache(), many third-party output plugins could
have their own caching mechanisms, making it impossible for reorderbuffer to
simulate cache management for all. Therefore, what the reorderbuffer should do,
in my opinion, would be to directly ask the output plugin to see if a result
can be obtained without entering a transaction, as per the interface I
mentioned earlier.

[1]: /messages/by-id/CAGfChW7XpZGkxOpHZnv69+H_AyKzbffcrumyNu4Fz0u+1ADPxA@mail.gmail.com

Best Regards,
Hou zj

#11Ajin Cherian
itsajin@gmail.com
In reply to: Zhijie Hou (Fujitsu) (#10)
3 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Fri, Jan 24, 2025 at 5:28 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:

Hi,

I think the proposed feature is interesting, and after reviewing the
original discussion thread, I'd like to help move it forward. To that end,
I've
summarized the key points of the discussion so far, including the
challenges
highlighted in the thread and some issues I identified off-list, along with
potential solutions. My colleague will share a new POC patch set that has
been
developed based on these lines.
(Note that the previous email was blocked, this is the second attempt)

Thanks Hou-san,

Here's a patch-set created based on the design changes proposed by Hou-san.

P.S: It has reached v12 after internal review/rework iterations

Regards,
Ajin Cherian
Fujitsu Australia

Attachments:

v12-0001-Track-transactions-with-internal-snapshot-change.patchapplication/octet-stream; name=v12-0001-Track-transactions-with-internal-snapshot-change.patchDownload
From 269184738e6b5b5e888bb4dd44d7f3ebec6151cf Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Fri, 24 Jan 2025 00:11:29 -0500
Subject: [PATCH v12 1/3] Track transactions with internal snapshot changes

Track transactions which have snapshot changes with a new flag RBTXN_HAS_SNAPSHOT_CHANGES
---
 src/backend/replication/logical/reorderbuffer.c | 11 +++++++++++
 src/include/replication/reorderbuffer.h         |  7 +++++++
 2 files changed, 18 insertions(+)

diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 79b60df..121a1c2 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -823,6 +823,14 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 
 		toptxn->txn_flags |= RBTXN_HAS_STREAMABLE_CHANGE;
 	}
+	else if (change->action == REORDER_BUFFER_CHANGE_INVALIDATION ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID)
+	{
+		ReorderBufferTXN *toptxn = rbtxn_get_toptxn(txn);
+
+		toptxn->txn_flags |= RBTXN_HAS_SNAPSHOT_CHANGES;
+	}
 
 	change->lsn = lsn;
 	change->txn = txn;
@@ -1747,6 +1755,9 @@ ReorderBufferTruncateTXN(ReorderBuffer *rb, ReorderBufferTXN *txn, bool txn_prep
 		txn->txn_flags |= RBTXN_IS_SERIALIZED_CLEAR;
 	}
 
+	/* All snapshot changes up to this point have been processed. */
+	txn->txn_flags &= ~RBTXN_HAS_SNAPSHOT_CHANGES;
+
 	/* also reset the number of entries in the transaction */
 	txn->nentries_mem = 0;
 	txn->nentries = 0;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index a669658..0329a69 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -173,6 +173,7 @@ typedef struct ReorderBufferChange
 #define RBTXN_PREPARE             	0x0040
 #define RBTXN_SKIPPED_PREPARE	  	0x0080
 #define RBTXN_HAS_STREAMABLE_CHANGE	0x0100
+#define RBTXN_HAS_SNAPSHOT_CHANGES	0x0200
 
 /* Does the transaction have catalog changes? */
 #define rbtxn_has_catalog_changes(txn) \
@@ -210,6 +211,12 @@ typedef struct ReorderBufferChange
 	((txn)->txn_flags & RBTXN_HAS_STREAMABLE_CHANGE) != 0 \
 )
 
+/* Does this transaction have snapshot changes? */
+#define rbtxn_has_snapshot_changes(txn) \
+( \
+	((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
+)
+
 /*
  * Has this transaction been streamed to downstream?
  *
-- 
1.8.3.1

v12-0002-Filter-transactions-that-need-not-be-published.patchapplication/octet-stream; name=v12-0002-Filter-transactions-that-need-not-be-published.patchDownload
From e6827ae016721819e682596e5370b98e595dfaa1 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Fri, 24 Jan 2025 00:30:55 -0500
Subject: [PATCH v12 2/3] Filter transactions that need not be published

This adds logic to filter transactions early (at decode time, rather than at streaming time) so
they can be skipped if they do not contain tables that are part of the publication list of the
logical replication walsender.

Determining whether to filter a change requires information about the relation
and the publication from the catalog, requiring a transaction to be started.
When most changes in a transaction are unfilterable, the overhead of starting a
transaction for each record is significant. To reduce this overhead a hash cache of relation file
locators is created. Even then a hash search for every record before recording has considerable
overhead especially for use cases where most tables in an instance are published. To further reduce
this overhead a simple approach is used to suspend filtering for a certain number of changes
(100) when an unfilterable change is encountered. In other words, continue filtering changes if
the last record was filtered out. If an unfilterable change is found, skip filtering the next 100
changes.
---
 src/backend/replication/logical/decode.c        |   5 +
 src/backend/replication/logical/reorderbuffer.c | 423 ++++++++++++++++++++----
 src/include/replication/reorderbuffer.h         |  12 +
 src/tools/pgindent/typedefs.list                |   2 +
 4 files changed, 374 insertions(+), 68 deletions(-)

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 0bff0f1..c5f1083 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	if (ctx->reorder->can_filter_change &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator, true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	if (!(xlrec->flags & XLH_INSERT_IS_SPECULATIVE))
 		change->action = REORDER_BUFFER_CHANGE_INSERT;
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 121a1c2..5974c77 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -108,6 +108,7 @@
 #include "storage/fd.h"
 #include "storage/sinval.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 #include "utils/relfilenumbermap.h"
@@ -226,8 +227,10 @@ static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
 											   XLogRecPtr lsn, bool create_as_top);
 static void ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
 											  ReorderBufferTXN *subtxn);
-
 static void AssertTXNLsnOrder(ReorderBuffer *rb);
+static Relation ReorderBufferGetRelation(ReorderBuffer *rb,
+										 RelFileLocator *rlocator,
+										 bool has_tuple);
 
 /* ---------------------------------------
  * support functions for lsn-order iterating over the ->changes of a
@@ -276,6 +279,8 @@ static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
  */
 static inline bool ReorderBufferCanStream(ReorderBuffer *rb);
 static inline bool ReorderBufferCanStartStreaming(ReorderBuffer *rb);
+static Snapshot ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb,
+											   ReorderBufferTXN *txn);
 static void ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
 static void ReorderBufferStreamCommit(ReorderBuffer *rb, ReorderBufferTXN *txn);
 
@@ -302,6 +307,48 @@ static void ReorderBufferChangeMemoryUpdate(ReorderBuffer *rb,
 											bool addition, Size sz);
 
 /*
+ * ---------------------------------------
+ * RelFileLocator filtering
+ * ---------------------------------------
+ * This hash table serves as a lookup table for determining if a relation can
+ * be filtered before being decoded and queued into the buffer.
+ *
+ * The hash table shares the same lifespan as the reorder buffer. This is
+ * crucial because each reorderbuffer may have different configurations or be
+ * associated with different output plugins, affecting the types of tables to
+ * be processed.
+ */
+
+static HTAB *RelFileLocatorFilterCache = NULL;
+
+static bool relation_callbacks_registered = false;
+
+typedef struct ReorderBufferRelFileLocatorKey
+{
+	Oid			reltablespace;
+	RelFileNumber relfilenumber;
+} ReorderBufferRelFileLocatorKey;
+
+/* entry for hash table we use to track if the relation can be filtered */
+typedef struct ReorderBufferRelFileLocatorEnt
+{
+	ReorderBufferRelFileLocatorKey key;
+	Oid			relid;
+	bool		filterable;
+} ReorderBufferRelFileLocatorEnt;
+
+static void RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid);
+static void ReorderBufferMemoryResetcallback(void *arg);
+
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100
+
+/*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
  * prior ReorderBuffer instances for the same slot.
  */
@@ -359,6 +406,34 @@ ReorderBufferAllocate(void)
 	buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
 								 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
+	/*
+	 * To support early filtering of changes, this hash table acts
+	 * as a lookup table to determine if the corresponding relation is
+	 * required to be decoded and queued into the buffer. The hash table
+	 * shares the same lifespan as the reorder buffer.
+	 *
+	 * Also setup the callback to invalidate cache when relations are updated.
+	 */
+	hash_ctl.keysize = sizeof(ReorderBufferRelFileLocatorKey);
+	hash_ctl.entrysize = sizeof(ReorderBufferRelFileLocatorEnt);
+	hash_ctl.hcxt = buffer->context;
+
+	RelFileLocatorFilterCache =
+		hash_create("RelFileLocatorFilterCache", 1000, &hash_ctl,
+					HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	buffer->relfile_callback.arg = NULL;
+	buffer->relfile_callback.func = ReorderBufferMemoryResetcallback;
+	MemoryContextRegisterResetCallback(buffer->context, &buffer->relfile_callback);
+
+	if (!relation_callbacks_registered)
+	{
+		/* Watch for invalidation events. */
+		CacheRegisterRelcacheCallback(RelFileLocatorCacheInvalidateCallback,
+									  (Datum) 0);
+		relation_callbacks_registered = true;
+	}
+
 	buffer->by_txn_last_xid = InvalidTransactionId;
 	buffer->by_txn_last_txn = NULL;
 
@@ -369,6 +444,8 @@ ReorderBufferAllocate(void)
 	/* txn_heap is ordered by transaction size */
 	buffer->txn_heap = pairingheap_allocate(ReorderBufferTXNSizeCompare, NULL);
 
+	buffer->can_filter_change = true;
+
 	buffer->spillTxns = 0;
 	buffer->spillCount = 0;
 	buffer->spillBytes = 0;
@@ -840,6 +917,16 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 	txn->nentries++;
 	txn->nentries_mem++;
 
+	/*
+	 * If we're not filtering and we've crossed the change threshold,
+	 * attempt to filter again
+	 */
+	if (!rb->can_filter_change && ++rb->processed_changes >= CHANGES_THRESHOLD_FOR_FILTER)
+	{
+		rb->can_filter_change = true;
+		rb->processed_changes = 0;
+	}
+
 	/* update memory accounting information */
 	ReorderBufferChangeMemoryUpdate(rb, change, NULL, true,
 									ReorderBufferChangeSize(change));
@@ -2105,6 +2192,81 @@ ReorderBufferResetTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 }
 
 /*
+ * Return the relation corresponding to the given RelFileLocator 'rlocator' if
+ * relevant to decoding. Otherwise, return NULL.
+ *
+ * We don't decode catalog files, relations that are not logically logged,
+ * temporary heaps and sequences.
+ */
+static Relation
+ReorderBufferGetRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+						 bool has_tuple)
+{
+	bool		filterable = false;
+	Relation	relation;
+	Oid			reloid;
+
+	reloid = RelidByRelfilenumber(rlocator->spcOid, rlocator->relNumber);
+
+	if (!OidIsValid(reloid))
+	{
+		if (!has_tuple)
+		{
+			/*
+			 * Mapped catalog tuple without data, emitted while catalog table was in
+			 * the process of being rewritten. We can fail to look up the
+			 * relfilenumber, because the relmapper has no "historic" view, in
+			 * contrast to the normal catalog during decoding. Thus repeated rewrites
+			 * can cause a lookup failure. That's OK because we do not decode catalog
+			 * changes anyway. Normally such tuples would be skipped over below, but
+			 * we can't identify whether the table should be logically logged without
+			 * mapping the relfilenumber to the oid.
+			 */
+			return NULL;
+		}
+
+		elog(ERROR, "could not map filenumber \"%s\" to relation OID",
+				relpathperm(*rlocator, MAIN_FORKNUM));
+	}
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid, relpathperm(*rlocator, MAIN_FORKNUM));
+
+	if (!RelationIsLogicallyLogged(relation))
+		filterable = true;
+
+	else if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		/*
+		 * Ignore temporary heaps created during DDL unless the plugin has asked
+		 * for them.
+		 */
+		filterable = true;
+	}
+
+	else if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		/*
+		 * For now ignore sequence changes entirely. Most of the time they don't
+		 * log changes using records we understand, so it doesn't make sense to
+		 * handle the few cases we do.
+		 */
+		filterable = true;
+	}
+
+	if (filterable)
+	{
+		RelationClose(relation);
+		return NULL;
+	}
+
+	return relation;
+}
+
+/*
  * Helper function for ReorderBufferReplay and ReorderBufferStreamTXN.
  *
  * Send data of a transaction (and its subtransactions) to the
@@ -2176,7 +2338,6 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 		while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
 		{
 			Relation	relation = NULL;
-			Oid			reloid;
 
 			CHECK_FOR_INTERRUPTS();
 
@@ -2235,55 +2396,11 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 				case REORDER_BUFFER_CHANGE_DELETE:
 					Assert(snapshot_now);
 
-					reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
-												  change->data.tp.rlocator.relNumber);
-
-					/*
-					 * Mapped catalog tuple without data, emitted while
-					 * catalog table was in the process of being rewritten. We
-					 * can fail to look up the relfilenumber, because the
-					 * relmapper has no "historic" view, in contrast to the
-					 * normal catalog during decoding. Thus repeated rewrites
-					 * can cause a lookup failure. That's OK because we do not
-					 * decode catalog changes anyway. Normally such tuples
-					 * would be skipped over below, but we can't identify
-					 * whether the table should be logically logged without
-					 * mapping the relfilenumber to the oid.
-					 */
-					if (reloid == InvalidOid &&
-						change->data.tp.newtuple == NULL &&
-						change->data.tp.oldtuple == NULL)
-						goto change_done;
-					else if (reloid == InvalidOid)
-						elog(ERROR, "could not map filenumber \"%s\" to relation OID",
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM));
-
-					relation = RelationIdGetRelation(reloid);
+					relation = ReorderBufferGetRelation(rb, &change->data.tp.rlocator,
+														(change->data.tp.newtuple != NULL ||
+														 change->data.tp.oldtuple != NULL));
 
 					if (!RelationIsValid(relation))
-						elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
-							 reloid,
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM));
-
-					if (!RelationIsLogicallyLogged(relation))
-						goto change_done;
-
-					/*
-					 * Ignore temporary heaps created during DDL unless the
-					 * plugin has asked for them.
-					 */
-					if (relation->rd_rel->relrewrite && !rb->output_rewrites)
-						goto change_done;
-
-					/*
-					 * For now ignore sequence changes entirely. Most of the
-					 * time they don't log changes using records we
-					 * understand, so it doesn't make sense to handle the few
-					 * cases we do.
-					 */
-					if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
 						goto change_done;
 
 					/* user-triggered change */
@@ -4059,19 +4176,13 @@ ReorderBufferCanStartStreaming(ReorderBuffer *rb)
 }
 
 /*
- * Send data of a large transaction (and its subtransactions) to the
- * output plugin, but using the stream API.
+ * This function generates or retrieves a consistent snapshot of a transaction
+ * that is currently in progress for logical replication.
  */
-static void
-ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+static Snapshot
+ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb, ReorderBufferTXN *txn)
 {
 	Snapshot	snapshot_now;
-	CommandId	command_id;
-	Size		stream_bytes;
-	bool		txn_is_streamed;
-
-	/* We can never reach here for a subtransaction. */
-	Assert(rbtxn_is_toptxn(txn));
 
 	/*
 	 * We can't make any assumptions about base snapshot here, similar to what
@@ -4114,12 +4225,11 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		if (txn->base_snapshot == NULL)
 		{
 			Assert(txn->ninvalidations == 0);
-			return;
+			return NULL;
 		}
 
-		command_id = FirstCommandId;
 		snapshot_now = ReorderBufferCopySnap(rb, txn->base_snapshot,
-											 txn, command_id);
+											 txn, FirstCommandId);
 	}
 	else
 	{
@@ -4132,17 +4242,40 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		 * the LSN condition in the previous branch (so no need to walk
 		 * through subxacts again). In fact, we must not do that as we may be
 		 * using snapshot half-way through the subxact.
-		 */
-		command_id = txn->command_id;
-
-		/*
+		 *
 		 * We can't use txn->snapshot_now directly because after the last
 		 * streaming run, we might have got some new sub-transactions. So we
 		 * need to add them to the snapshot.
 		 */
 		snapshot_now = ReorderBufferCopySnap(rb, txn->snapshot_now,
-											 txn, command_id);
+											 txn, txn->command_id);
+	}
+
+	return snapshot_now;
+}
+
+/*
+ * Send data of a large transaction (and its subtransactions) to the
+ * output plugin, but using the stream API.
+ */
+static void
+ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+	Snapshot	snapshot_now;
+	Size		stream_bytes;
+	bool		txn_is_streamed;
+
+	/* We can never reach here for a subtransaction. */
+	Assert(rbtxn_is_toptxn(txn));
+
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, txn);
+
+	/* This transaction didn't make any changes to the database till now. */
+	if (snapshot_now == NULL)
+		return;
 
+	if (txn->snapshot_now != NULL)
+	{
 		/* Free the previously copied snapshot. */
 		Assert(txn->snapshot_now->copied);
 		ReorderBufferFreeSnap(rb, txn->snapshot_now);
@@ -4160,7 +4293,7 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 
 	/* Process and send the changes to output plugin. */
 	ReorderBufferProcessTXN(rb, txn, InvalidXLogRecPtr, snapshot_now,
-							command_id, true);
+							snapshot_now->curcid, true);
 
 	rb->streamCount += 1;
 	rb->streamBytes += stream_bytes;
@@ -5349,3 +5482,157 @@ restart:
 		*cmax = ent->cmax;
 	return true;
 }
+
+/*
+ * Flush mapping entries when the corresponding relations are updated in pg_class.
+ */
+static void
+RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid)
+{
+	HASH_SEQ_STATUS status;
+	ReorderBufferRelFileLocatorEnt *entry;
+
+	if (!RelFileLocatorFilterCache)
+		return;
+
+	hash_seq_init(&status, RelFileLocatorFilterCache);
+
+	/* slightly inefficient algorithm but not performance critical path */
+	while ((entry = (ReorderBufferRelFileLocatorEnt *) hash_seq_search(&status)) != NULL)
+	{
+		/*
+		 * If relid is InvalidOid, signaling a complete reset, we must remove
+		 * all entries, otherwise just remove the specific relation's entry.
+		 * Always remove negative cache entries.
+		 */
+		if (relid == InvalidOid ||	/* complete reset */
+			entry->relid == InvalidOid ||	/* negative cache entry */
+			entry->relid == relid)	/* individual flushed relation */
+		{
+			if (hash_search(RelFileLocatorFilterCache,
+							&entry->key,
+							HASH_REMOVE,
+							NULL) == NULL)
+				elog(ERROR, "hash table corrupted");
+		}
+	}
+}
+
+/*
+ * Context reset/delete callback for RelFileLocatorFilterCache.
+ */
+static void
+ReorderBufferMemoryResetcallback(void *arg)
+{
+	RelFileLocatorFilterCache = NULL;
+}
+
+/*
+ * Determine whether the record corresponding to the relation identified by
+ * 'rlocator' needs to be filtered and not decoded and queued in the buffer.
+ *
+ * Determining the necessity of decoding requires accessing relation
+ * information from the system catalog, which requires a historical snapshot.
+ * We cannot directly use the current snapshot of the transaction due to the
+ * presence of INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION records in the
+ * buffer that could modify the snapshot. A proper snapshot can only be
+ * constructed after these records are processed in ReorderBufferProcessTXN, or
+ * if we are decoding a transaction without these records. See comment on top
+ * of ReorderBufferGetRelation() to see list of relations that are not
+ * decoded by the reorderbuffer.
+ *
+ * To reduce the overhead from the system catalog access and transaction
+ * management, the results are cached in the 'RelFileLocatorFilterCache' hash
+ * table.
+ *
+ * Returns true if the relation can be filtered; otherwise, false.
+ */
+bool
+ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+									XLogRecPtr lsn, RelFileLocator *rlocator,
+									bool has_tuple)
+{
+	bool		found;
+	Relation	relation;
+	bool		using_subtxn;
+	Snapshot	snapshot_now;
+	ReorderBufferTXN *txn,
+			   *toptxn;
+	ReorderBufferRelFileLocatorEnt *entry;
+	ReorderBufferRelFileLocatorKey key;
+
+	Assert(RelFileLocatorFilterCache);
+
+	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+	Assert(txn);
+	toptxn = rbtxn_get_toptxn(txn);
+	rb->can_filter_change = false;
+
+	/*
+	 * We cannot construct an accurate historical snapshot until all pending
+	 * records in this transaction that might update the snapshot are
+	 * processed.
+	 */
+	if (rbtxn_has_snapshot_changes(toptxn))
+		return false;
+
+	key.reltablespace = rlocator->spcOid;
+	key.relfilenumber = rlocator->relNumber;
+	entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+	if (found)
+	{
+		rb->can_filter_change = entry->filterable;
+		return entry->filterable;
+	}
+
+	/* constructs a temporary historical snapshot */
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, toptxn);
+	Assert(snapshot_now);
+
+	/* build data to be able to lookup the CommandIds of catalog tuples */
+	ReorderBufferBuildTupleCidHash(rb, toptxn);
+
+	/* setup the initial snapshot */
+	SetupHistoricSnapshot(snapshot_now, toptxn->tuplecid_hash);
+
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by RelFileLocator");
+	else
+		StartTransactionCommand();
+
+	relation = ReorderBufferGetRelation(rb, rlocator, has_tuple);
+
+	if (RelationIsValid(relation))
+	{
+		entry->relid = RelationGetRelid(relation);
+		entry->filterable = false;
+		RelationClose(relation);
+	}
+	else
+	{
+		entry->relid = InvalidOid;
+		entry->filterable = true;
+	}
+
+	rb->can_filter_change = entry->filterable;
+
+	ReorderBufferFreeSnap(rb, snapshot_now);
+
+	TeardownHistoricSnapshot(false);
+
+	/*
+	 * Aborting the current (sub-)transaction as a whole has the right
+	 * semantics. We want all locks acquired in here to be released, not
+	 * reassigned to the parent and we do not want any database access have
+	 * persistent effects.
+	 */
+	AbortCurrentTransaction();
+
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	return entry->filterable;
+}
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 0329a69..b1267e9 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -622,6 +622,7 @@ struct ReorderBuffer
 	 * Private memory context.
 	 */
 	MemoryContext context;
+	MemoryContextCallback relfile_callback;
 
 	/*
 	 * Memory contexts for specific types objects
@@ -642,6 +643,12 @@ struct ReorderBuffer
 	/* Max-heap for sizes of all top-level and sub transactions */
 	pairingheap *txn_heap;
 
+	/* should we try to filter the change? */
+	bool		can_filter_change;
+
+	/* number of changes after a failed attempt at filtering */
+	int8		processed_changes;
+
 	/*
 	 * Statistics about transactions spilled to disk.
 	 *
@@ -742,4 +749,9 @@ extern void ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr);
 
 extern void StartupReorderBuffer(void);
 
+extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+												XLogRecPtr lsn, RelFileLocator *rlocator,
+												bool has_tuple);
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
+
 #endif
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index a2644a2..0feca52 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2433,6 +2433,8 @@ ReorderBufferIterTXNEntry
 ReorderBufferIterTXNState
 ReorderBufferMessageCB
 ReorderBufferPrepareCB
+ReorderBufferRelFileLocatorEnt
+ReorderBufferRelFileLocatorKey
 ReorderBufferRollbackPreparedCB
 ReorderBufferStreamAbortCB
 ReorderBufferStreamChangeCB
-- 
1.8.3.1

v12-0003-Introduce-a-output-plugin-callback-to-filter-cha.patchapplication/octet-stream; name=v12-0003-Introduce-a-output-plugin-callback-to-filter-cha.patchDownload
From 4f3f09618e91d9a0e36bfa3163a64384df6ee4d2 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Fri, 24 Jan 2025 00:40:53 -0500
Subject: [PATCH v12 3/3] Introduce a output plugin callback to filter changes

This new output plugin callback provides an option to logical decoding plugins to filter out
changes early. The primary purpose of the callback is to conserve memory and processing cycles
by excluding changes that are not required by output plugins.
---
 doc/src/sgml/logicaldecoding.sgml               | 41 ++++++++++++++++++-
 src/backend/replication/logical/decode.c        | 22 +++++++++-
 src/backend/replication/logical/logical.c       | 42 +++++++++++++++++++
 src/backend/replication/logical/reorderbuffer.c | 45 +++++++++++++++++----
 src/backend/replication/pgoutput/pgoutput.c     | 54 +++++++++++++++++++++++++
 src/include/replication/output_plugin.h         | 20 +++++++++
 src/include/replication/reorderbuffer.h         |  8 ++++
 src/test/subscription/t/013_partition.pl        |  7 ++++
 8 files changed, 228 insertions(+), 11 deletions(-)

diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 1c4ae38..a603c47 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -560,6 +560,7 @@ typedef struct OutputPluginCallbacks
     LogicalDecodeCommitCB commit_cb;
     LogicalDecodeMessageCB message_cb;
     LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+    LogicalDecodeFilterChangeCB filter_change_cb;
     LogicalDecodeShutdownCB shutdown_cb;
     LogicalDecodeFilterPrepareCB filter_prepare_cb;
     LogicalDecodeBeginPrepareCB begin_prepare_cb;
@@ -582,8 +583,8 @@ typedef void (*LogicalOutputPluginInit) (struct OutputPluginCallbacks *cb);
      and <function>commit_cb</function> callbacks are required,
      while <function>startup_cb</function>, <function>truncate_cb</function>,
      <function>message_cb</function>, <function>filter_by_origin_cb</function>,
-     and <function>shutdown_cb</function> are optional.
-     If <function>truncate_cb</function> is not set but a
+     <function>shutdown_cb</function>, and <function>filter_change_cb</function>
+     are optional. If <function>truncate_cb</function> is not set but a
      <command>TRUNCATE</command> is to be decoded, the action will be ignored.
     </para>
 
@@ -871,6 +872,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
      </para>
      </sect3>
 
+     <sect3 id="logicaldecoding-output-plugin-filter-change">
+     <title>Change Filter Callback</title>
+
+     <para>
+      The optional <function>filter_change_cb</function> is called before a
+      change record is decoded to determine whether the change can be filtered
+      out.
+<programlisting>
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+                                             Oid relid,
+                                             ReorderBufferChangeType change_type,
+                                             bool in_txn, bool *cache_valid);
+</programlisting>
+      To indicate that decoding can be skipped for the given change
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.
+      The <parameter>in_txn</parameter> parameter indicates whether the
+      callback is invoked within a transaction block.
+      When <parameter>in_txn</parameter> is false, and if making a decision to filter a change requires being inside a
+      transaction block, such as needing access to the catalog, set
+      <parameter>*cache_valid</parameter> to <literal>false</literal>.
+      This ensures that the callback will be reinvoked once a transaction block
+      starts. If a decision can be made immediately, set
+      <parameter>*cache_valid</parameter> to <literal>true</literal>.
+     </para>
+     <para>
+      The primary purpose of this callback function is to optimize memory usage
+      and processing efficiency by filtering out changes that are unnecessary for
+      output plugins. It enables output plugins to selectively process relevant
+      changes. Caching filtering decisions locally is recommended, as it enables
+      the callback to provide cached results without repeatedly initiating
+      transactions or querying the catalog. This approach minimizes overhead
+      and improves efficiency during the decoding process.
+     </para>
+     </sect3>
+
     <sect3 id="logicaldecoding-output-plugin-message">
      <title>Generic Message Callback</title>
 
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index c5f1083..7c01e7c 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -576,6 +576,17 @@ FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
 }
 
 /*
+ * Check if filtering changes before decoding is supported and we're not suppressing filter
+ * changes currently.
+ */
+static inline bool
+FilterChangeIsEnabled(LogicalDecodingContext *ctx)
+{
+	return (ctx->callbacks.filter_change_cb != NULL &&
+				ctx->reorder->can_filter_change);
+}
+
+/*
  * Handle rmgr LOGICALMSG_ID records for LogicalDecodingProcessRecord().
  */
 void
@@ -915,9 +926,16 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
-	if (ctx->reorder->can_filter_change &&
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
 		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
-											buf->origptr, &target_locator, true))
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_INSERT,
+											true))
 		return;
 
 	change = ReorderBufferGetChange(ctx->reorder);
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 0b25efa..adc5383 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -74,6 +74,9 @@ static void truncate_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 							   XLogRecPtr message_lsn, bool transactional,
 							   const char *prefix, Size message_size, const char *message);
+static bool filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+								  ReorderBufferChangeType change_type, bool in_txn,
+								  bool *cache_valid);
 
 /* streaming callbacks */
 static void stream_start_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
@@ -220,6 +223,7 @@ StartupDecodingContext(List *output_plugin_options,
 	/* wrap output plugin callbacks, so we can add error context information */
 	ctx->reorder->begin = begin_cb_wrapper;
 	ctx->reorder->apply_change = change_cb_wrapper;
+	ctx->reorder->filter_change = filter_change_cb_wrapper;
 	ctx->reorder->apply_truncate = truncate_cb_wrapper;
 	ctx->reorder->commit = commit_cb_wrapper;
 	ctx->reorder->message = message_cb_wrapper;
@@ -1243,6 +1247,44 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+static bool
+filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+					  ReorderBufferChangeType change_type, bool in_txn,
+					  bool *cache_valid)
+{
+	LogicalDecodingContext *ctx = cache->private_data;
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	/* check if the filter change callback is supported */
+	if (ctx->callbacks.filter_change_cb == NULL)
+		return false;
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_change";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_change_cb(ctx, relid, change_type, in_txn, cache_valid);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 5974c77..052d91f 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -5550,9 +5550,11 @@ ReorderBufferMemoryResetcallback(void *arg)
 bool
 ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 									XLogRecPtr lsn, RelFileLocator *rlocator,
+									ReorderBufferChangeType change_type,
 									bool has_tuple)
 {
 	bool		found;
+	bool		cache_valid;
 	Relation	relation;
 	bool		using_subtxn;
 	Snapshot	snapshot_now;
@@ -5566,7 +5568,6 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
 	Assert(txn);
 	toptxn = rbtxn_get_toptxn(txn);
-	rb->can_filter_change = false;
 
 	/*
 	 * We cannot construct an accurate historical snapshot until all pending
@@ -5583,7 +5584,25 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (found)
 	{
 		rb->can_filter_change = entry->filterable;
-		return entry->filterable;
+
+		/*
+		 * Quick return if we already know that the relation is not to be decoded.
+		 * These are for special relations that are unlogged and for sequences
+		 * and catalogs.
+		 */
+		if (entry->filterable)
+			return true;
+
+		/* Allow the output plugin to filter relations */
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  false, &cache_valid);
+
+		/*
+		 * If plugin had the relation ready in cache, the response is valid, else
+		 * we'll need to call the plugin a second time within a transaction.
+		 */
+		if (cache_valid)
+			return rb->can_filter_change;
 	}
 
 	/* constructs a temporary historical snapshot */
@@ -5607,18 +5626,30 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 
 	if (RelationIsValid(relation))
 	{
-		entry->relid = RelationGetRelid(relation);
+		if (IsToastRelation(relation))
+		{
+			Oid     real_reloid = InvalidOid;
+			char   *toast_name = RelationGetRelationName(relation);
+			/* pg_toast_ len is 9 */
+			char   *start_ch = &toast_name[9];
+
+			real_reloid = pg_strtoint32(start_ch);
+			entry->relid = real_reloid;
+		}
+		else
+			entry->relid = RelationGetRelid(relation);
+
 		entry->filterable = false;
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  true, &cache_valid);
 		RelationClose(relation);
 	}
 	else
 	{
 		entry->relid = InvalidOid;
-		entry->filterable = true;
+		rb->can_filter_change = entry->filterable = true;
 	}
 
-	rb->can_filter_change = entry->filterable;
-
 	ReorderBufferFreeSnap(rb, snapshot_now);
 
 	TeardownHistoricSnapshot(false);
@@ -5634,5 +5665,5 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (using_subtxn)
 		RollbackAndReleaseCurrentSubTransaction();
 
-	return entry->filterable;
+	return rb->can_filter_change;
 }
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index a363c88..c15b053 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -56,6 +56,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+								   ReorderBufferChangeType change_type, bool in_txn,
+								   bool *cache_valid);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -267,6 +270,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_change_cb = pgoutput_filter_change;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1742,6 +1746,56 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 }
 
 /*
+ * Determine whether a change to the specified relation should be published.
+ *
+ * See the comments atop of LogicalDecodeFilterChangeCB for details.
+ */
+static bool
+pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+					   ReorderBufferChangeType change_type, bool in_txn, bool *cache_valid)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *entry;
+
+	Assert(RelationSyncCache != NULL);
+
+	if (in_txn)
+	{
+		Relation	relation;
+
+		relation = RelationIdGetRelation(relid);
+		entry = get_rel_sync_entry(data, relation);
+		*cache_valid = true;
+	}
+	else
+	{
+		entry = (RelationSyncEntry *) hash_search(RelationSyncCache,
+												  &relid,
+												  HASH_FIND, cache_valid);
+		if (!*cache_valid)
+			return false;
+	}
+
+	/*
+	 * If the pubaction is not supported by this publication then return true to say
+	 * the change for this entry can be skipped.
+	 */
+	switch (change_type)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			return !entry->pubactions.pubinsert;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			return !entry->pubactions.pubupdate;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			return !entry->pubactions.pubdelete;
+		default:
+			/* allow any other changes that are not explicitly filtered */
+			return false;
+	}
+
+}
+
+/*
  * Shutdown the output plugin.
  *
  * Note, we don't need to clean the data->context, data->cachectx, and
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 8d4d5b71..f82bc1d 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -97,6 +97,25 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
 											   RepOriginId origin_id);
 
 /*
+ * This callback is called before a change record is decoded to determine
+ * whether the change can be filtered out before entering the reorder buffer.
+ *
+ * Typically, the output plugin needs to refer to the system catalog to make
+ * this decision. To enhance efficiency, the plugin should cache the lookup
+ * result for each relation, minimizing catalog access on subsequent calls.
+ *
+ * If the callback is called with 'in_txn' set as false (the reorder
+ * buffer has not started a transaction), the output plugin should set '*cache_valid'
+ * to false to indicate that the result is not available in its internal cache.
+ * If 'in_txn' is true, the plugin can create a cache entry after querying the
+ * catalog.
+ */
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
+/*
  * Called to shutdown an output plugin.
  */
 typedef void (*LogicalDecodeShutdownCB) (struct LogicalDecodingContext *ctx);
@@ -222,6 +241,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterChangeCB filter_change_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b1267e9..b138365 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -468,6 +468,12 @@ typedef void (*ReorderBufferMessageCB) (ReorderBuffer *rb,
 										const char *prefix, Size sz,
 										const char *message);
 
+/* filter change callback signature */
+typedef bool (*ReorderBufferFilterChangeCB) (ReorderBuffer *rb,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
 /* begin prepare callback signature */
 typedef void (*ReorderBufferBeginPrepareCB) (ReorderBuffer *rb,
 											 ReorderBufferTXN *txn);
@@ -578,6 +584,7 @@ struct ReorderBuffer
 	 */
 	ReorderBufferBeginCB begin;
 	ReorderBufferApplyChangeCB apply_change;
+	ReorderBufferFilterChangeCB filter_change;
 	ReorderBufferApplyTruncateCB apply_truncate;
 	ReorderBufferCommitCB commit;
 	ReorderBufferMessageCB message;
@@ -751,6 +758,7 @@ extern void StartupReorderBuffer(void);
 
 extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 												XLogRecPtr lsn, RelFileLocator *rlocator,
+												ReorderBufferChangeType change_type,
 												bool has_tuple);
 extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
 
diff --git a/src/test/subscription/t/013_partition.pl b/src/test/subscription/t/013_partition.pl
index 14a3bea..180eaa0 100644
--- a/src/test/subscription/t/013_partition.pl
+++ b/src/test/subscription/t/013_partition.pl
@@ -468,6 +468,13 @@ $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4 (a int PRIMARY KEY)");
 $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4_1 (a int PRIMARY KEY)");
+
+# Ensure that the subscription 'sub2' catches up with the latest changes. This
+# is necessary for the walsender to update its historic snapshot. Otherwise,
+# the walsender might retain an outdated snapshot, potentially preventing it
+# from accessing the newly created publication.
+$node_publisher->wait_for_catchup('sub2');
+
 # Since we specified publish_via_partition_root in pub_all and
 # pub_lower_level, all partition tables use their root tables' identity and
 # schema. We set the list of publications so that the FOR ALL TABLES
-- 
1.8.3.1

#12Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#11)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Some review comments for patch v12-0001.

======
Commit message

1.
Track transactions which have snapshot changes with a new flag
RBTXN_HAS_SNAPSHOT_CHANGES

~

The commit message only says *what* it does, but not *why* this patch
even exists. TBH, I don't understand why this patch needs to be
separated from your patch 0002, because 0001 makes no independent use
of the flag, nor is it separately tested.

Anyway, if it is going to remain separated then IMO at least the the
message should explain the intended purpose e.g. why the subsequent
patches require this flagged info and how they will use it.

======
src/include/replication/reorderbuffer.h

2.
+/* Does this transaction have snapshot changes? */
+#define rbtxn_has_snapshot_changes(txn) \
+( \
+ ((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
+)
+

Is the below wording maybe a more plain way to say that:

/* Does this transaction make changes to the current snapshot? */

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#13Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Ajin Cherian (#11)
1 attachment(s)
RE: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Dear Ajin, Hou,

- Snapshot construction

I understand the approach that we do not try to filter for all the workloads; do
just best-effort.

I played with your PoC and here are my comments.

1.
Can you add tests for better understanding? I've tried for tes_decoding like attached,
but it couldn't pass the regression test. Cases "stream.sql" and "twophase_stream.sql"
were failed.

2.
I think we can extend the skip mechanism to UPDATE/DELETE/MultiInsert/SpecConfirm.
Regarding the TRUNCATE, I'm not sure we can handle hte TRUNCATE case because the we
can't track RelFileLocator anymore.

3.
Both ReorderBuffer and RelFileLocatorFilterCache have the same lifetime but
RelFileLocatorFilterCache is provided as the global variable; it is quite strange.
Can you somehow avoid this? I considered idea but could not because we do not have
APIs to Unregister the relcacheCallback.

4.
For output plugins which does not cache the catalog, it may not able to do filtering
without the transaction. For them, the early filter may degrade the performance
because it requires to open transactions for every changes.
Is my understanding correct?

Best regards,
Hayato Kuroda
FUJITSU LIMITED

Attachments:

v20250131-0001-Try-to-add-tests.txttext/plain; name=v20250131-0001-Try-to-add-tests.txtDownload
From 1fc88208de13fae8d287d456b899d4ebe01558be Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Fri, 31 Jan 2025 21:45:31 +0900
Subject: [PATCH v20250131] Try to add tests

---
 contrib/test_decoding/Makefile            |  2 +-
 contrib/test_decoding/expected/filter.out | 26 +++++++++++++++++++++++
 contrib/test_decoding/meson.build         |  1 +
 contrib/test_decoding/sql/filter.sql      | 16 ++++++++++++++
 contrib/test_decoding/test_decoding.c     | 21 ++++++++++++++++++
 5 files changed, 65 insertions(+), 1 deletion(-)
 create mode 100644 contrib/test_decoding/expected/filter.out
 create mode 100644 contrib/test_decoding/sql/filter.sql

diff --git a/contrib/test_decoding/Makefile b/contrib/test_decoding/Makefile
index a4ba1a509a..a66ab2cbf8 100644
--- a/contrib/test_decoding/Makefile
+++ b/contrib/test_decoding/Makefile
@@ -5,7 +5,7 @@ PGFILEDESC = "test_decoding - example of a logical decoding output plugin"
 
 REGRESS = ddl xact rewrite toast permissions decoding_in_xact \
 	decoding_into_rel binary prepared replorigin time messages \
-	spill slot truncate stream stats twophase twophase_stream
+	spill slot truncate stream stats twophase twophase_stream filter
 ISOLATION = mxact delayed_startup ondisk_startup concurrent_ddl_dml \
 	oldest_xmin snapshot_transfer subxact_without_top concurrent_stream \
 	twophase_snapshot slot_creation_error catalog_change_snapshot \
diff --git a/contrib/test_decoding/expected/filter.out b/contrib/test_decoding/expected/filter.out
new file mode 100644
index 0000000000..a3b43f2772
--- /dev/null
+++ b/contrib/test_decoding/expected/filter.out
@@ -0,0 +1,26 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+ ?column? 
+----------
+ init
+(1 row)
+
+-- Create two tables
+CREATE TABLE test(id int);
+CREATE TABLE test_skipped(id int);
+-- Changes for XXX_skipped are skipped
+BEGIN;
+INSERT INTO test_skipped VALUES (1);
+COMMIT;
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
+ data 
+------
+(0 rows)
+
+SELECT pg_drop_replication_slot('regression_slot');
+ pg_drop_replication_slot 
+--------------------------
+ 
+(1 row)
+
diff --git a/contrib/test_decoding/meson.build b/contrib/test_decoding/meson.build
index 54d65d3f30..9f1f98e6e2 100644
--- a/contrib/test_decoding/meson.build
+++ b/contrib/test_decoding/meson.build
@@ -41,6 +41,7 @@ tests += {
       'stats',
       'twophase',
       'twophase_stream',
+      'filter',
     ],
     'regress_args': [
       '--temp-config', files('logical.conf'),
diff --git a/contrib/test_decoding/sql/filter.sql b/contrib/test_decoding/sql/filter.sql
new file mode 100644
index 0000000000..08ec80eb03
--- /dev/null
+++ b/contrib/test_decoding/sql/filter.sql
@@ -0,0 +1,16 @@
+-- predictability
+SET synchronous_commit = on;
+SELECT 'init' FROM pg_create_logical_replication_slot('regression_slot', 'test_decoding');
+
+-- Create two tables
+CREATE TABLE test(id int);
+CREATE TABLE test_skipped(id int);
+
+-- Changes for XXX_skipped are skipped
+BEGIN;
+INSERT INTO test_skipped VALUES (1);
+COMMIT;
+
+SELECT data FROM pg_logical_slot_get_changes('regression_slot', NULL, NULL, 'include-xids', '0', 'skip-empty-xacts', '1');
+
+SELECT pg_drop_replication_slot('regression_slot');
diff --git a/contrib/test_decoding/test_decoding.c b/contrib/test_decoding/test_decoding.c
index 0113b19636..789a4bc2aa 100644
--- a/contrib/test_decoding/test_decoding.c
+++ b/contrib/test_decoding/test_decoding.c
@@ -68,6 +68,9 @@ static void pg_decode_truncate(LogicalDecodingContext *ctx,
 							   ReorderBufferChange *change);
 static bool pg_decode_filter(LogicalDecodingContext *ctx,
 							 RepOriginId origin_id);
+static bool pg_decode_filter_change(LogicalDecodingContext *ctx, Oid relid,
+									ReorderBufferChangeType change_type,
+									bool in_txn, bool *cache_valid);
 static void pg_decode_message(LogicalDecodingContext *ctx,
 							  ReorderBufferTXN *txn, XLogRecPtr lsn,
 							  bool transactional, const char *prefix,
@@ -133,6 +136,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->truncate_cb = pg_decode_truncate;
 	cb->commit_cb = pg_decode_commit_txn;
 	cb->filter_by_origin_cb = pg_decode_filter;
+	cb->filter_change_cb = pg_decode_filter_change;
 	cb->shutdown_cb = pg_decode_shutdown;
 	cb->message_cb = pg_decode_message;
 	cb->filter_prepare_cb = pg_decode_filter_prepare;
@@ -467,6 +471,23 @@ pg_decode_filter(LogicalDecodingContext *ctx,
 	return false;
 }
 
+static bool
+pg_decode_filter_change(LogicalDecodingContext *ctx, Oid relid,
+						ReorderBufferChangeType change_type, bool in_txn,
+						bool *cache_valid)
+{
+	if (in_txn)
+	{
+		Relation	relation = RelationIdGetRelation(relid);
+		char	   *relname = NameStr(relation->rd_rel->relname);
+
+		if (strstr(relname, "_skipped") != NULL)
+			return true;
+	}
+
+	return false;
+}
+
 /*
  * Print literal `outputstr' already represented as string of type `typid'
  * into stringbuf `s'.
-- 
2.43.5

#14vignesh C
vignesh21@gmail.com
In reply to: Ajin Cherian (#11)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Tue, 28 Jan 2025 at 08:40, Ajin Cherian <itsajin@gmail.com> wrote:

Here's a patch-set created based on the design changes proposed by Hou-san.

Few comments:
1) Shouldn't we do the same thing for other DecodeXXX functions?
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx,
XLogRecordBuffer *buf)
if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
return;

+       if (ctx->reorder->can_filter_change &&
+               ReorderBufferFilterByRelFileLocator(ctx->reorder,
XLogRecGetXid(r),
+
                 buf->origptr, &target_locator, true))
+               return;
+

2) Let's add some debug logs so that it will be easy to verify the
changes that are getting filtered, or else we will have to debug and
verify them:
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx,
XLogRecordBuffer *buf)
if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
return;

+       if (ctx->reorder->can_filter_change &&
+               ReorderBufferFilterByRelFileLocator(ctx->reorder,
XLogRecGetXid(r),
+
                 buf->origptr, &target_locator, true))
+               return;

3) Also there are no tests currently, probably if we add the above
mentioned debug logs we could add few tests and verify them based on
the logs.

4) Can you elaborate a bit in the commit message why we need to
capture if a transaction has snapshot changes:
Subject: [PATCH v12 1/3] Track transactions with internal snapshot changes

Track transactions which have snapshot changes with a new flag
RBTXN_HAS_SNAPSHOT_CHANGES

Regards,
Vignesh

#15Zhijie Hou (Fujitsu)
houzj.fnst@fujitsu.com
In reply to: Hayato Kuroda (Fujitsu) (#13)
RE: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Friday, January 31, 2025 9:43 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com>

Dear Ajin, Hou,

- Snapshot construction

I understand the approach that we do not try to filter for all the workloads; do
just best-effort.

3.
Both ReorderBuffer and RelFileLocatorFilterCache have the same lifetime but
RelFileLocatorFilterCache is provided as the global variable; it is quite strange.
Can you somehow avoid this? I considered idea but could not because we do
not have APIs to Unregister the relcacheCallback.

I think we already have precedent of this (e.g., RelationSyncCache in pgoutput.c),
so, I am OK for it unless we come up with better ideas.

4.
For output plugins which does not cache the catalog, it may not able to do
filtering without the transaction. For them, the early filter may degrade the
performance because it requires to open transactions for every changes.
Is my understanding correct?

Partially correct. To be precise, I think we should recommend that output
plugin that cannot filter without transaction can avoid implementing this
callback in the first place, so that it would not cause degradation.

Best Regards,
Hou zj

#16Ajin Cherian
itsajin@gmail.com
In reply to: vignesh C (#14)
3 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Wed, Jan 29, 2025 at 9:31 AM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v12-0001.

======
Commit message

1.
Track transactions which have snapshot changes with a new flag
RBTXN_HAS_SNAPSHOT_CHANGES

~

The commit message only says *what* it does, but not *why* this patch
even exists. TBH, I don't understand why this patch needs to be
separated from your patch 0002, because 0001 makes no independent use
of the flag, nor is it separately tested.

Anyway, if it is going to remain separated then IMO at least the the
message should explain the intended purpose e.g. why the subsequent
patches require this flagged info and how they will use it.

Fixed.

======
src/include/replication/reorderbuffer.h

    2.
    +/* Does this transaction have snapshot changes? */
    +#define rbtxn_has_snapshot_changes(txn) \
    +( \
    + ((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
    +)
    +

Is the below wording maybe a more plain way to say that:

/* Does this transaction make changes to the current snapshot? */

Fixed as suggested.

On Sat, Feb 1, 2025 at 12:43 AM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:
Dear Ajin, Hou,

- Snapshot construction

I understand the approach that we do not try to filter for all the workloads; do
just best-effort.

I played with your PoC and here are my comments.

1.
Can you add tests for better understanding? I've tried for
tes_decoding like attached,
but it couldn't pass the regression test. Cases "stream.sql" and
"twophase_stream.sql"
were failed.

I've added tests. I've analysed your patch and the regression test
failure. The tests fail to detect concurrent aborts.I think the reason
for that is because of the filter check logic access of the catalog,
the changes are cached.
As a result, when the actual decoding happens, the catalog is not
accessed as the relation detailsare in cache. Without catalog access,
concurrent aborts cannot be detected as concurrent aborts are
detectedonly when the catalog is accessed. There is a new thread by
Sawada-san on a more efficient detection of concurrent aborts,I don't
know if that will solve this issue, otherwise I don't know how to fix
this in a meaningful way. Caching improvesperformance, and at the same
time it prevents detection of concurrent aborts.

2.
I think we can extend the skip mechanism to
UPDATE/DELETE/MultiInsert/SpecConfirm.
Regarding the TRUNCATE, I'm not sure we can handle hte TRUNCATE case
because the we
can't track RelFileLocator anymore.

Updated.

On Tue, Feb 4, 2025 at 2:19 PM vignesh C <vignesh21@gmail.com> wrote:

On Tue, 28 Jan 2025 at 08:40, Ajin Cherian <itsajin@gmail.com> wrote:

Here's a patch-set created based on the design changes proposed

by Hou-san.

Few comments:
1) Shouldn't we do the same thing for other DecodeXXX functions?
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx,
XLogRecordBuffer *buf)
if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
return;

    +       if (ctx->reorder->can_filter_change &&
    +               ReorderBufferFilterByRelFileLocator(ctx->reorder,
    XLogRecGetXid(r),
    +
                     buf->origptr, &target_locator, true))
    +               return;
    +

Updated.

2) Let's add some debug logs so that it will be easy to verify the
changes that are getting filtered, or else we will have to debug and
verify them:
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx,
XLogRecordBuffer *buf)
if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
return;

    +       if (ctx->reorder->can_filter_change &&
    +               ReorderBufferFilterByRelFileLocator(ctx->reorder,
    XLogRecGetXid(r),
    +
                     buf->origptr, &target_locator, true))
    +               return;

3) Also there are no tests currently, probably if we add the above
mentioned debug logs we could add few tests and verify them based on
the logs.

Updated.

4) Can you elaborate a bit in the commit message why we need to
capture if a transaction has snapshot changes:
Subject: [PATCH v12 1/3] Track transactions with internal snapshot changes

Track transactions which have snapshot changes with a new flag
RBTXN_HAS_SNAPSHOT_CHANGES

Updated.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

v13-0003-Introduce-a-output-plugin-callback-to-filter-cha.patchapplication/octet-stream; name=v13-0003-Introduce-a-output-plugin-callback-to-filter-cha.patchDownload
From 944993771427e94d8e74411303c876b61d757bc9 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Tue, 11 Feb 2025 23:49:50 -0500
Subject: [PATCH v13 3/3] Introduce a output plugin callback to filter changes

This new output plugin callback provides an option to logical decoding plugins to filter out
changes early. The primary purpose of the callback is to conserve memory and processing cycles
by excluding changes that are not required by output plugins.
---
 doc/src/sgml/logicaldecoding.sgml               | 41 ++++++++++++++-
 src/backend/replication/logical/decode.c        | 70 ++++++++++++++++++++++++-
 src/backend/replication/logical/logical.c       | 42 +++++++++++++++
 src/backend/replication/logical/reorderbuffer.c | 45 +++++++++++++---
 src/backend/replication/pgoutput/pgoutput.c     | 70 +++++++++++++++++++++++++
 src/include/replication/output_plugin.h         | 20 +++++++
 src/include/replication/reorderbuffer.h         |  8 +++
 src/test/subscription/t/001_rep_changes.pl      | 18 ++++++-
 src/test/subscription/t/013_partition.pl        |  7 +++
 9 files changed, 308 insertions(+), 13 deletions(-)

diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 1c4ae38..a603c47 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -560,6 +560,7 @@ typedef struct OutputPluginCallbacks
     LogicalDecodeCommitCB commit_cb;
     LogicalDecodeMessageCB message_cb;
     LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+    LogicalDecodeFilterChangeCB filter_change_cb;
     LogicalDecodeShutdownCB shutdown_cb;
     LogicalDecodeFilterPrepareCB filter_prepare_cb;
     LogicalDecodeBeginPrepareCB begin_prepare_cb;
@@ -582,8 +583,8 @@ typedef void (*LogicalOutputPluginInit) (struct OutputPluginCallbacks *cb);
      and <function>commit_cb</function> callbacks are required,
      while <function>startup_cb</function>, <function>truncate_cb</function>,
      <function>message_cb</function>, <function>filter_by_origin_cb</function>,
-     and <function>shutdown_cb</function> are optional.
-     If <function>truncate_cb</function> is not set but a
+     <function>shutdown_cb</function>, and <function>filter_change_cb</function>
+     are optional. If <function>truncate_cb</function> is not set but a
      <command>TRUNCATE</command> is to be decoded, the action will be ignored.
     </para>
 
@@ -871,6 +872,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
      </para>
      </sect3>
 
+     <sect3 id="logicaldecoding-output-plugin-filter-change">
+     <title>Change Filter Callback</title>
+
+     <para>
+      The optional <function>filter_change_cb</function> is called before a
+      change record is decoded to determine whether the change can be filtered
+      out.
+<programlisting>
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+                                             Oid relid,
+                                             ReorderBufferChangeType change_type,
+                                             bool in_txn, bool *cache_valid);
+</programlisting>
+      To indicate that decoding can be skipped for the given change
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.
+      The <parameter>in_txn</parameter> parameter indicates whether the
+      callback is invoked within a transaction block.
+      When <parameter>in_txn</parameter> is false, and if making a decision to filter a change requires being inside a
+      transaction block, such as needing access to the catalog, set
+      <parameter>*cache_valid</parameter> to <literal>false</literal>.
+      This ensures that the callback will be reinvoked once a transaction block
+      starts. If a decision can be made immediately, set
+      <parameter>*cache_valid</parameter> to <literal>true</literal>.
+     </para>
+     <para>
+      The primary purpose of this callback function is to optimize memory usage
+      and processing efficiency by filtering out changes that are unnecessary for
+      output plugins. It enables output plugins to selectively process relevant
+      changes. Caching filtering decisions locally is recommended, as it enables
+      the callback to provide cached results without repeatedly initiating
+      transactions or querying the catalog. This approach minimizes overhead
+      and improves efficiency during the decoding process.
+     </para>
+     </sect3>
+
     <sect3 id="logicaldecoding-output-plugin-message">
      <title>Generic Message Callback</title>
 
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index c5f1083..fd6a177 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -576,6 +576,17 @@ FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
 }
 
 /*
+ * Check if filtering changes before decoding is supported and we're not suppressing filter
+ * changes currently.
+ */
+static inline bool
+FilterChangeIsEnabled(LogicalDecodingContext *ctx)
+{
+	return (ctx->callbacks.filter_change_cb != NULL &&
+				ctx->reorder->can_filter_change);
+}
+
+/*
  * Handle rmgr LOGICALMSG_ID records for LogicalDecodingProcessRecord().
  */
 void
@@ -915,9 +926,16 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
-	if (ctx->reorder->can_filter_change &&
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
 		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
-											buf->origptr, &target_locator, true))
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_INSERT,
+											true))
 		return;
 
 	change = ReorderBufferGetChange(ctx->reorder);
@@ -970,6 +988,18 @@ DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_UPDATE,
+											true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_UPDATE;
 	change->origin_id = XLogRecGetOrigin(r);
@@ -1036,6 +1066,18 @@ DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_DELETE,
+											true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 
 	if (xlrec->flags & XLH_DELETE_IS_SUPER)
@@ -1139,6 +1181,18 @@ DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 		return;
 
 	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &rlocator,
+											REORDER_BUFFER_CHANGE_INSERT,
+											true))
+		return;
+
+	/*
 	 * We know that this multi_insert isn't for a catalog, so the block should
 	 * always have data even if a full-page write of it is taken.
 	 */
@@ -1231,6 +1285,18 @@ DecodeSpecConfirm(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_INSERT,
+											true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM;
 	change->origin_id = XLogRecGetOrigin(r);
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 8ea846b..635e1de 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -74,6 +74,9 @@ static void truncate_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 							   XLogRecPtr message_lsn, bool transactional,
 							   const char *prefix, Size message_size, const char *message);
+static bool filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+								  ReorderBufferChangeType change_type, bool in_txn,
+								  bool *cache_valid);
 
 /* streaming callbacks */
 static void stream_start_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
@@ -220,6 +223,7 @@ StartupDecodingContext(List *output_plugin_options,
 	/* wrap output plugin callbacks, so we can add error context information */
 	ctx->reorder->begin = begin_cb_wrapper;
 	ctx->reorder->apply_change = change_cb_wrapper;
+	ctx->reorder->filter_change = filter_change_cb_wrapper;
 	ctx->reorder->apply_truncate = truncate_cb_wrapper;
 	ctx->reorder->commit = commit_cb_wrapper;
 	ctx->reorder->message = message_cb_wrapper;
@@ -1224,6 +1228,44 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+static bool
+filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+					  ReorderBufferChangeType change_type, bool in_txn,
+					  bool *cache_valid)
+{
+	LogicalDecodingContext *ctx = cache->private_data;
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	/* check if the filter change callback is supported */
+	if (ctx->callbacks.filter_change_cb == NULL)
+		return false;
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_change";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_change_cb(ctx, relid, change_type, in_txn, cache_valid);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 7be0033..9a1dfb6 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -5549,9 +5549,11 @@ ReorderBufferMemoryResetcallback(void *arg)
 bool
 ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 									XLogRecPtr lsn, RelFileLocator *rlocator,
+									ReorderBufferChangeType change_type,
 									bool has_tuple)
 {
 	bool		found;
+	bool		cache_valid;
 	Relation	relation;
 	bool		using_subtxn;
 	Snapshot	snapshot_now;
@@ -5565,7 +5567,6 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
 	Assert(txn);
 	toptxn = rbtxn_get_toptxn(txn);
-	rb->can_filter_change = false;
 
 	/*
 	 * We cannot construct an accurate historical snapshot until all pending
@@ -5582,7 +5583,25 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (found)
 	{
 		rb->can_filter_change = entry->filterable;
-		return entry->filterable;
+
+		/*
+		 * Quick return if we already know that the relation is not to be decoded.
+		 * These are for special relations that are unlogged and for sequences
+		 * and catalogs.
+		 */
+		if (entry->filterable)
+			return true;
+
+		/* Allow the output plugin to filter relations */
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  false, &cache_valid);
+
+		/*
+		 * If plugin had the relation ready in cache, the response is valid, else
+		 * we'll need to call the plugin a second time within a transaction.
+		 */
+		if (cache_valid)
+			return rb->can_filter_change;
 	}
 
 	/* constructs a temporary historical snapshot */
@@ -5606,18 +5625,30 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 
 	if (RelationIsValid(relation))
 	{
-		entry->relid = RelationGetRelid(relation);
+		if (IsToastRelation(relation))
+		{
+			Oid     real_reloid = InvalidOid;
+			char   *toast_name = RelationGetRelationName(relation);
+			/* pg_toast_ len is 9 */
+			char   *start_ch = &toast_name[9];
+
+			real_reloid = pg_strtoint32(start_ch);
+			entry->relid = real_reloid;
+		}
+		else
+			entry->relid = RelationGetRelid(relation);
+
 		entry->filterable = false;
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  true, &cache_valid);
 		RelationClose(relation);
 	}
 	else
 	{
 		entry->relid = InvalidOid;
-		entry->filterable = true;
+		rb->can_filter_change = entry->filterable = true;
 	}
 
-	rb->can_filter_change = entry->filterable;
-
 	ReorderBufferFreeSnap(rb, snapshot_now);
 
 	TeardownHistoricSnapshot(false);
@@ -5633,5 +5664,5 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (using_subtxn)
 		RollbackAndReleaseCurrentSubTransaction();
 
-	return entry->filterable;
+	return rb->can_filter_change;
 }
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index 7d464f6..93eb579 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -57,6 +57,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+								   ReorderBufferChangeType change_type, bool in_txn,
+								   bool *cache_valid);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -268,6 +271,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_change_cb = pgoutput_filter_change;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1744,6 +1748,72 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 }
 
 /*
+ * Determine whether a change to the specified relation should be published.
+ *
+ * See the comments atop of LogicalDecodeFilterChangeCB for details.
+ */
+static bool
+pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+					   ReorderBufferChangeType change_type, bool in_txn, bool *cache_valid)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *entry;
+
+	Assert(RelationSyncCache != NULL);
+
+	if (in_txn)
+	{
+		Relation	relation;
+
+		relation = RelationIdGetRelation(relid);
+		entry = get_rel_sync_entry(data, relation);
+		*cache_valid = true;
+	}
+	else
+	{
+		entry = (RelationSyncEntry *) hash_search(RelationSyncCache,
+												  &relid,
+												  HASH_FIND, cache_valid);
+		if (!*cache_valid)
+			return false;
+	}
+
+	/*
+	 * If the pubaction is not supported by this publication then return true to say
+	 * the change for this entry can be skipped.
+	 */
+	switch (change_type)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			if (!entry->pubactions.pubinsert)
+			{
+				elog(DEBUG1, "Filtering INSERT");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			if (!entry->pubactions.pubupdate)
+			{
+				elog(DEBUG1, "Filtering UPDATE");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			if (!entry->pubactions.pubdelete)
+			{
+				elog(DEBUG1, "Filtering DELETE");
+				return true;
+			}
+			break;
+		default:
+			/* allow any other changes that are not explicitly filtered */
+			return false;
+	}
+
+	return false;
+}
+
+/*
  * Shutdown the output plugin.
  *
  * Note, we don't need to clean the data->context, data->cachectx, and
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 8d4d5b71..f82bc1d 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -97,6 +97,25 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
 											   RepOriginId origin_id);
 
 /*
+ * This callback is called before a change record is decoded to determine
+ * whether the change can be filtered out before entering the reorder buffer.
+ *
+ * Typically, the output plugin needs to refer to the system catalog to make
+ * this decision. To enhance efficiency, the plugin should cache the lookup
+ * result for each relation, minimizing catalog access on subsequent calls.
+ *
+ * If the callback is called with 'in_txn' set as false (the reorder
+ * buffer has not started a transaction), the output plugin should set '*cache_valid'
+ * to false to indicate that the result is not available in its internal cache.
+ * If 'in_txn' is true, the plugin can create a cache entry after querying the
+ * catalog.
+ */
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
+/*
  * Called to shutdown an output plugin.
  */
 typedef void (*LogicalDecodeShutdownCB) (struct LogicalDecodingContext *ctx);
@@ -222,6 +241,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterChangeCB filter_change_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 7d805d4..0b83ef4 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -468,6 +468,12 @@ typedef void (*ReorderBufferMessageCB) (ReorderBuffer *rb,
 										const char *prefix, Size sz,
 										const char *message);
 
+/* filter change callback signature */
+typedef bool (*ReorderBufferFilterChangeCB) (ReorderBuffer *rb,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
 /* begin prepare callback signature */
 typedef void (*ReorderBufferBeginPrepareCB) (ReorderBuffer *rb,
 											 ReorderBufferTXN *txn);
@@ -578,6 +584,7 @@ struct ReorderBuffer
 	 */
 	ReorderBufferBeginCB begin;
 	ReorderBufferApplyChangeCB apply_change;
+	ReorderBufferFilterChangeCB filter_change;
 	ReorderBufferApplyTruncateCB apply_truncate;
 	ReorderBufferCommitCB commit;
 	ReorderBufferMessageCB message;
@@ -751,6 +758,7 @@ extern void StartupReorderBuffer(void);
 
 extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 												XLogRecPtr lsn, RelFileLocator *rlocator,
+												ReorderBufferChangeType change_type,
 												bool has_tuple);
 extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
 
diff --git a/src/test/subscription/t/001_rep_changes.pl b/src/test/subscription/t/001_rep_changes.pl
index 8726fe0..bced62c 100644
--- a/src/test/subscription/t/001_rep_changes.pl
+++ b/src/test/subscription/t/001_rep_changes.pl
@@ -485,9 +485,10 @@ $node_publisher->wait_for_catchup('tap_sub');
 
 # Check that we don't send BEGIN and COMMIT because of empty transaction
 # optimization.  We have to look for the DEBUG1 log messages about that, so
-# temporarily bump up the log verbosity.
+# temporarily bump up the log verbosity. Also confirm that unpublished
+# changes are filtered out after a restart.
 $node_publisher->append_conf('postgresql.conf', "log_min_messages = debug1");
-$node_publisher->reload;
+$node_publisher->restart;
 
 # Note that the current location of the log file is not grabbed immediately
 # after reloading the configuration, but after sending one SQL command to
@@ -495,6 +496,8 @@ $node_publisher->reload;
 $log_location = -s $node_publisher->logfile;
 
 $node_publisher->safe_psql('postgres', "INSERT INTO tab_notrep VALUES (11)");
+$node_publisher->safe_psql('postgres', "UPDATE tab_notrep SET a = 2 WHERE a = 1");
+$node_publisher->safe_psql('postgres', "DELETE FROM tab_notrep WHERE a = 2");
 
 $node_publisher->wait_for_catchup('tap_sub');
 
@@ -502,6 +505,17 @@ $logfile = slurp_file($node_publisher->logfile, $log_location);
 ok($logfile =~ qr/skipped replication of an empty transaction with XID/,
 	'empty transaction is skipped');
 
+# Check that an unpublished change is filtered out.
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+
+ok($logfile =~ qr/Filtering UPDATE/,
+	'unpublished UPDATE is filtered');
+
+ok($logfile =~ qr/Filtering DELETE/,
+	'unpublished DELETE is filtered');
+
 $result =
   $node_subscriber->safe_psql('postgres', "SELECT count(*) FROM tab_notrep");
 is($result, qq(0), 'check non-replicated table is empty on subscriber');
diff --git a/src/test/subscription/t/013_partition.pl b/src/test/subscription/t/013_partition.pl
index 14a3bea..180eaa0 100644
--- a/src/test/subscription/t/013_partition.pl
+++ b/src/test/subscription/t/013_partition.pl
@@ -468,6 +468,13 @@ $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4 (a int PRIMARY KEY)");
 $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4_1 (a int PRIMARY KEY)");
+
+# Ensure that the subscription 'sub2' catches up with the latest changes. This
+# is necessary for the walsender to update its historic snapshot. Otherwise,
+# the walsender might retain an outdated snapshot, potentially preventing it
+# from accessing the newly created publication.
+$node_publisher->wait_for_catchup('sub2');
+
 # Since we specified publish_via_partition_root in pub_all and
 # pub_lower_level, all partition tables use their root tables' identity and
 # schema. We set the list of publications so that the FOR ALL TABLES
-- 
1.8.3.1

v13-0001-Track-transactions-with-internal-snapshot-change.patchapplication/octet-stream; name=v13-0001-Track-transactions-with-internal-snapshot-change.patchDownload
From fec9567d30f333fb1b4b94bca4b7289146beadbc Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Tue, 11 Feb 2025 03:25:34 -0500
Subject: [PATCH v13 1/3] Track transactions with internal snapshot changes

Track transactions that make changes to the current snapshot with a new flag
RBTXN_HAS_SNAPSHOT_CHANGES. This will allow logical decoding to accumulate changes
which modify the snapshot before creating a historic snaphot for a transaction.
---
 src/backend/replication/logical/reorderbuffer.c | 11 +++++++++++
 src/include/replication/reorderbuffer.h         |  7 +++++++
 2 files changed, 18 insertions(+)

diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 10a3766..3d22aa9 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -823,6 +823,14 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 
 		toptxn->txn_flags |= RBTXN_HAS_STREAMABLE_CHANGE;
 	}
+	else if (change->action == REORDER_BUFFER_CHANGE_INVALIDATION ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID)
+	{
+		ReorderBufferTXN *toptxn = rbtxn_get_toptxn(txn);
+
+		toptxn->txn_flags |= RBTXN_HAS_SNAPSHOT_CHANGES;
+	}
 
 	change->lsn = lsn;
 	change->txn = txn;
@@ -1747,6 +1755,9 @@ ReorderBufferTruncateTXN(ReorderBuffer *rb, ReorderBufferTXN *txn, bool txn_prep
 		txn->txn_flags |= RBTXN_IS_SERIALIZED_CLEAR;
 	}
 
+	/* All snapshot changes up to this point have been processed. */
+	txn->txn_flags &= ~RBTXN_HAS_SNAPSHOT_CHANGES;
+
 	/* also reset the number of entries in the transaction */
 	txn->nentries_mem = 0;
 	txn->nentries = 0;
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index a669658..b0f26c7 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -173,6 +173,7 @@ typedef struct ReorderBufferChange
 #define RBTXN_PREPARE             	0x0040
 #define RBTXN_SKIPPED_PREPARE	  	0x0080
 #define RBTXN_HAS_STREAMABLE_CHANGE	0x0100
+#define RBTXN_HAS_SNAPSHOT_CHANGES	0x0200
 
 /* Does the transaction have catalog changes? */
 #define rbtxn_has_catalog_changes(txn) \
@@ -210,6 +211,12 @@ typedef struct ReorderBufferChange
 	((txn)->txn_flags & RBTXN_HAS_STREAMABLE_CHANGE) != 0 \
 )
 
+/* Does this transaction make changes to the current snapshot? */
+#define rbtxn_has_snapshot_changes(txn) \
+( \
+	((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
+)
+
 /*
  * Has this transaction been streamed to downstream?
  *
-- 
1.8.3.1

v13-0002-Filter-transactions-that-need-not-be-published.patchapplication/octet-stream; name=v13-0002-Filter-transactions-that-need-not-be-published.patchDownload
From cb1e99a1bea946f4da3f8ae6b0068d2464adfafa Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Tue, 11 Feb 2025 04:07:21 -0500
Subject: [PATCH v13 2/3] Filter transactions that need not be published

This adds logic to filter transactions early (at decode time, rather than at streaming time) so
they can be skipped if they do not contain tables that are part of the publication list of the
logical replication walsender.

Determining whether to filter a change requires information about the relation
and the publication from the catalog, requiring a transaction to be started.
When most changes in a transaction are unfilterable, the overhead of starting a
transaction for each record is significant. To reduce this overhead a hash cache of relation file
locators is created. Even then a hash search for every record before recording has considerable
overhead especially for use cases where most tables in an instance are published. To further reduce
this overhead a simple approach is used to suspend filtering for a certain number of changes
(100) when an unfilterable change is encountered. In other words, continue filtering changes if
the last record was filtered out. If an unfilterable change is found, skip filtering the next 100
changes.
---
 src/backend/replication/logical/decode.c        |   5 +
 src/backend/replication/logical/reorderbuffer.c | 423 ++++++++++++++++++++----
 src/include/replication/reorderbuffer.h         |  12 +
 src/tools/pgindent/typedefs.list                |   2 +
 4 files changed, 374 insertions(+), 68 deletions(-)

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 0bff0f1..c5f1083 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	if (ctx->reorder->can_filter_change &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator, true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	if (!(xlrec->flags & XLH_INSERT_IS_SPECULATIVE))
 		change->action = REORDER_BUFFER_CHANGE_INSERT;
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 3d22aa9..7be0033 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -108,6 +108,7 @@
 #include "storage/fd.h"
 #include "storage/sinval.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 #include "utils/relfilenumbermap.h"
@@ -226,8 +227,10 @@ static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
 											   XLogRecPtr lsn, bool create_as_top);
 static void ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
 											  ReorderBufferTXN *subtxn);
-
 static void AssertTXNLsnOrder(ReorderBuffer *rb);
+static Relation ReorderBufferGetRelation(ReorderBuffer *rb,
+										 RelFileLocator *rlocator,
+										 bool has_tuple);
 
 /* ---------------------------------------
  * support functions for lsn-order iterating over the ->changes of a
@@ -276,6 +279,8 @@ static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
  */
 static inline bool ReorderBufferCanStream(ReorderBuffer *rb);
 static inline bool ReorderBufferCanStartStreaming(ReorderBuffer *rb);
+static Snapshot ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb,
+											   ReorderBufferTXN *txn);
 static void ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
 static void ReorderBufferStreamCommit(ReorderBuffer *rb, ReorderBufferTXN *txn);
 
@@ -302,6 +307,48 @@ static void ReorderBufferChangeMemoryUpdate(ReorderBuffer *rb,
 											bool addition, Size sz);
 
 /*
+ * ---------------------------------------
+ * RelFileLocator filtering
+ * ---------------------------------------
+ * This hash table serves as a lookup table for determining if a relation can
+ * be filtered before being decoded and queued into the buffer.
+ *
+ * The hash table shares the same lifespan as the reorder buffer. This is
+ * crucial because each reorderbuffer may have different configurations or be
+ * associated with different output plugins, affecting the types of tables to
+ * be processed.
+ */
+
+static HTAB *RelFileLocatorFilterCache = NULL;
+
+static bool relation_callbacks_registered = false;
+
+typedef struct ReorderBufferRelFileLocatorKey
+{
+	Oid			reltablespace;
+	RelFileNumber relfilenumber;
+} ReorderBufferRelFileLocatorKey;
+
+/* entry for hash table we use to track if the relation can be filtered */
+typedef struct ReorderBufferRelFileLocatorEnt
+{
+	ReorderBufferRelFileLocatorKey key;
+	Oid			relid;
+	bool		filterable;
+} ReorderBufferRelFileLocatorEnt;
+
+static void RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid);
+static void ReorderBufferMemoryResetcallback(void *arg);
+
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100
+
+/*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
  * prior ReorderBuffer instances for the same slot.
  */
@@ -359,6 +406,34 @@ ReorderBufferAllocate(void)
 	buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
 								 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
+	/*
+	 * To support early filtering of changes, this hash table acts
+	 * as a lookup table to determine if the corresponding relation is
+	 * required to be decoded and queued into the buffer. The hash table
+	 * shares the same lifespan as the reorder buffer.
+	 *
+	 * Also setup the callback to invalidate cache when relations are updated.
+	 */
+	hash_ctl.keysize = sizeof(ReorderBufferRelFileLocatorKey);
+	hash_ctl.entrysize = sizeof(ReorderBufferRelFileLocatorEnt);
+	hash_ctl.hcxt = buffer->context;
+
+	RelFileLocatorFilterCache =
+		hash_create("RelFileLocatorFilterCache", 1000, &hash_ctl,
+					HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	buffer->relfile_callback.arg = NULL;
+	buffer->relfile_callback.func = ReorderBufferMemoryResetcallback;
+	MemoryContextRegisterResetCallback(buffer->context, &buffer->relfile_callback);
+
+	if (!relation_callbacks_registered)
+	{
+		/* Watch for invalidation events. */
+		CacheRegisterRelcacheCallback(RelFileLocatorCacheInvalidateCallback,
+									  (Datum) 0);
+		relation_callbacks_registered = true;
+	}
+
 	buffer->by_txn_last_xid = InvalidTransactionId;
 	buffer->by_txn_last_txn = NULL;
 
@@ -369,6 +444,8 @@ ReorderBufferAllocate(void)
 	/* txn_heap is ordered by transaction size */
 	buffer->txn_heap = pairingheap_allocate(ReorderBufferTXNSizeCompare, NULL);
 
+	buffer->can_filter_change = true;
+
 	buffer->spillTxns = 0;
 	buffer->spillCount = 0;
 	buffer->spillBytes = 0;
@@ -840,6 +917,16 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 	txn->nentries++;
 	txn->nentries_mem++;
 
+	/*
+	 * If we're not filtering and we've crossed the change threshold,
+	 * attempt to filter again
+	 */
+	if (!rb->can_filter_change && ++rb->processed_changes >= CHANGES_THRESHOLD_FOR_FILTER)
+	{
+		rb->can_filter_change = true;
+		rb->processed_changes = 0;
+	}
+
 	/* update memory accounting information */
 	ReorderBufferChangeMemoryUpdate(rb, change, NULL, true,
 									ReorderBufferChangeSize(change));
@@ -2105,6 +2192,81 @@ ReorderBufferResetTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 }
 
 /*
+ * Return the relation corresponding to the given RelFileLocator 'rlocator' if
+ * relevant to decoding. Otherwise, return NULL.
+ *
+ * We don't decode catalog files, relations that are not logically logged,
+ * temporary heaps and sequences.
+ */
+static Relation
+ReorderBufferGetRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+						 bool has_tuple)
+{
+	bool		filterable = false;
+	Relation	relation;
+	Oid			reloid;
+
+	reloid = RelidByRelfilenumber(rlocator->spcOid, rlocator->relNumber);
+
+	if (!OidIsValid(reloid))
+	{
+		if (!has_tuple)
+		{
+			/*
+			 * Mapped catalog tuple without data, emitted while catalog table was in
+			 * the process of being rewritten. We can fail to look up the
+			 * relfilenumber, because the relmapper has no "historic" view, in
+			 * contrast to the normal catalog during decoding. Thus repeated rewrites
+			 * can cause a lookup failure. That's OK because we do not decode catalog
+			 * changes anyway. Normally such tuples would be skipped over below, but
+			 * we can't identify whether the table should be logically logged without
+			 * mapping the relfilenumber to the oid.
+			 */
+			return NULL;
+		}
+
+		elog(ERROR, "could not map filenumber \"%s\" to relation OID",
+				relpathperm(*rlocator, MAIN_FORKNUM));
+	}
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid, relpathperm(*rlocator, MAIN_FORKNUM));
+
+	if (!RelationIsLogicallyLogged(relation))
+		filterable = true;
+
+	else if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		/*
+		 * Ignore temporary heaps created during DDL unless the plugin has asked
+		 * for them.
+		 */
+		filterable = true;
+	}
+
+	else if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		/*
+		 * For now ignore sequence changes entirely. Most of the time they don't
+		 * log changes using records we understand, so it doesn't make sense to
+		 * handle the few cases we do.
+		 */
+		filterable = true;
+	}
+
+	if (filterable)
+	{
+		RelationClose(relation);
+		return NULL;
+	}
+
+	return relation;
+}
+
+/*
  * Helper function for ReorderBufferReplay and ReorderBufferStreamTXN.
  *
  * Send data of a transaction (and its subtransactions) to the
@@ -2176,7 +2338,6 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 		while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
 		{
 			Relation	relation = NULL;
-			Oid			reloid;
 
 			CHECK_FOR_INTERRUPTS();
 
@@ -2235,55 +2396,11 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 				case REORDER_BUFFER_CHANGE_DELETE:
 					Assert(snapshot_now);
 
-					reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
-												  change->data.tp.rlocator.relNumber);
-
-					/*
-					 * Mapped catalog tuple without data, emitted while
-					 * catalog table was in the process of being rewritten. We
-					 * can fail to look up the relfilenumber, because the
-					 * relmapper has no "historic" view, in contrast to the
-					 * normal catalog during decoding. Thus repeated rewrites
-					 * can cause a lookup failure. That's OK because we do not
-					 * decode catalog changes anyway. Normally such tuples
-					 * would be skipped over below, but we can't identify
-					 * whether the table should be logically logged without
-					 * mapping the relfilenumber to the oid.
-					 */
-					if (reloid == InvalidOid &&
-						change->data.tp.newtuple == NULL &&
-						change->data.tp.oldtuple == NULL)
-						goto change_done;
-					else if (reloid == InvalidOid)
-						elog(ERROR, "could not map filenumber \"%s\" to relation OID",
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM));
-
-					relation = RelationIdGetRelation(reloid);
+					relation = ReorderBufferGetRelation(rb, &change->data.tp.rlocator,
+														(change->data.tp.newtuple != NULL ||
+														 change->data.tp.oldtuple != NULL));
 
 					if (!RelationIsValid(relation))
-						elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
-							 reloid,
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM));
-
-					if (!RelationIsLogicallyLogged(relation))
-						goto change_done;
-
-					/*
-					 * Ignore temporary heaps created during DDL unless the
-					 * plugin has asked for them.
-					 */
-					if (relation->rd_rel->relrewrite && !rb->output_rewrites)
-						goto change_done;
-
-					/*
-					 * For now ignore sequence changes entirely. Most of the
-					 * time they don't log changes using records we
-					 * understand, so it doesn't make sense to handle the few
-					 * cases we do.
-					 */
-					if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
 						goto change_done;
 
 					/* user-triggered change */
@@ -4058,19 +4175,13 @@ ReorderBufferCanStartStreaming(ReorderBuffer *rb)
 }
 
 /*
- * Send data of a large transaction (and its subtransactions) to the
- * output plugin, but using the stream API.
+ * This function generates or retrieves a consistent snapshot of a transaction
+ * that is currently in progress for logical replication.
  */
-static void
-ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+static Snapshot
+ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb, ReorderBufferTXN *txn)
 {
 	Snapshot	snapshot_now;
-	CommandId	command_id;
-	Size		stream_bytes;
-	bool		txn_is_streamed;
-
-	/* We can never reach here for a subtransaction. */
-	Assert(rbtxn_is_toptxn(txn));
 
 	/*
 	 * We can't make any assumptions about base snapshot here, similar to what
@@ -4113,12 +4224,11 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		if (txn->base_snapshot == NULL)
 		{
 			Assert(txn->ninvalidations == 0);
-			return;
+			return NULL;
 		}
 
-		command_id = FirstCommandId;
 		snapshot_now = ReorderBufferCopySnap(rb, txn->base_snapshot,
-											 txn, command_id);
+											 txn, FirstCommandId);
 	}
 	else
 	{
@@ -4131,17 +4241,40 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		 * the LSN condition in the previous branch (so no need to walk
 		 * through subxacts again). In fact, we must not do that as we may be
 		 * using snapshot half-way through the subxact.
-		 */
-		command_id = txn->command_id;
-
-		/*
+		 *
 		 * We can't use txn->snapshot_now directly because after the last
 		 * streaming run, we might have got some new sub-transactions. So we
 		 * need to add them to the snapshot.
 		 */
 		snapshot_now = ReorderBufferCopySnap(rb, txn->snapshot_now,
-											 txn, command_id);
+											 txn, txn->command_id);
+	}
+
+	return snapshot_now;
+}
+
+/*
+ * Send data of a large transaction (and its subtransactions) to the
+ * output plugin, but using the stream API.
+ */
+static void
+ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+	Snapshot	snapshot_now;
+	Size		stream_bytes;
+	bool		txn_is_streamed;
+
+	/* We can never reach here for a subtransaction. */
+	Assert(rbtxn_is_toptxn(txn));
+
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, txn);
+
+	/* This transaction didn't make any changes to the database till now. */
+	if (snapshot_now == NULL)
+		return;
 
+	if (txn->snapshot_now != NULL)
+	{
 		/* Free the previously copied snapshot. */
 		Assert(txn->snapshot_now->copied);
 		ReorderBufferFreeSnap(rb, txn->snapshot_now);
@@ -4159,7 +4292,7 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 
 	/* Process and send the changes to output plugin. */
 	ReorderBufferProcessTXN(rb, txn, InvalidXLogRecPtr, snapshot_now,
-							command_id, true);
+							snapshot_now->curcid, true);
 
 	rb->streamCount += 1;
 	rb->streamBytes += stream_bytes;
@@ -5348,3 +5481,157 @@ restart:
 		*cmax = ent->cmax;
 	return true;
 }
+
+/*
+ * Flush mapping entries when the corresponding relations are updated in pg_class.
+ */
+static void
+RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid)
+{
+	HASH_SEQ_STATUS status;
+	ReorderBufferRelFileLocatorEnt *entry;
+
+	if (!RelFileLocatorFilterCache)
+		return;
+
+	hash_seq_init(&status, RelFileLocatorFilterCache);
+
+	/* slightly inefficient algorithm but not performance critical path */
+	while ((entry = (ReorderBufferRelFileLocatorEnt *) hash_seq_search(&status)) != NULL)
+	{
+		/*
+		 * If relid is InvalidOid, signaling a complete reset, we must remove
+		 * all entries, otherwise just remove the specific relation's entry.
+		 * Always remove negative cache entries.
+		 */
+		if (relid == InvalidOid ||	/* complete reset */
+			entry->relid == InvalidOid ||	/* negative cache entry */
+			entry->relid == relid)	/* individual flushed relation */
+		{
+			if (hash_search(RelFileLocatorFilterCache,
+							&entry->key,
+							HASH_REMOVE,
+							NULL) == NULL)
+				elog(ERROR, "hash table corrupted");
+		}
+	}
+}
+
+/*
+ * Context reset/delete callback for RelFileLocatorFilterCache.
+ */
+static void
+ReorderBufferMemoryResetcallback(void *arg)
+{
+	RelFileLocatorFilterCache = NULL;
+}
+
+/*
+ * Determine whether the record corresponding to the relation identified by
+ * 'rlocator' needs to be filtered and not decoded and queued in the buffer.
+ *
+ * Determining the necessity of decoding requires accessing relation
+ * information from the system catalog, which requires a historical snapshot.
+ * We cannot directly use the current snapshot of the transaction due to the
+ * presence of INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION records in the
+ * buffer that could modify the snapshot. A proper snapshot can only be
+ * constructed after these records are processed in ReorderBufferProcessTXN, or
+ * if we are decoding a transaction without these records. See comment on top
+ * of ReorderBufferGetRelation() to see list of relations that are not
+ * decoded by the reorderbuffer.
+ *
+ * To reduce the overhead from the system catalog access and transaction
+ * management, the results are cached in the 'RelFileLocatorFilterCache' hash
+ * table.
+ *
+ * Returns true if the relation can be filtered; otherwise, false.
+ */
+bool
+ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+									XLogRecPtr lsn, RelFileLocator *rlocator,
+									bool has_tuple)
+{
+	bool		found;
+	Relation	relation;
+	bool		using_subtxn;
+	Snapshot	snapshot_now;
+	ReorderBufferTXN *txn,
+			   *toptxn;
+	ReorderBufferRelFileLocatorEnt *entry;
+	ReorderBufferRelFileLocatorKey key;
+
+	Assert(RelFileLocatorFilterCache);
+
+	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+	Assert(txn);
+	toptxn = rbtxn_get_toptxn(txn);
+	rb->can_filter_change = false;
+
+	/*
+	 * We cannot construct an accurate historical snapshot until all pending
+	 * records in this transaction that might update the snapshot are
+	 * processed.
+	 */
+	if (rbtxn_has_snapshot_changes(toptxn))
+		return false;
+
+	key.reltablespace = rlocator->spcOid;
+	key.relfilenumber = rlocator->relNumber;
+	entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+	if (found)
+	{
+		rb->can_filter_change = entry->filterable;
+		return entry->filterable;
+	}
+
+	/* constructs a temporary historical snapshot */
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, toptxn);
+	Assert(snapshot_now);
+
+	/* build data to be able to lookup the CommandIds of catalog tuples */
+	ReorderBufferBuildTupleCidHash(rb, toptxn);
+
+	/* setup the initial snapshot */
+	SetupHistoricSnapshot(snapshot_now, toptxn->tuplecid_hash);
+
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by RelFileLocator");
+	else
+		StartTransactionCommand();
+
+	relation = ReorderBufferGetRelation(rb, rlocator, has_tuple);
+
+	if (RelationIsValid(relation))
+	{
+		entry->relid = RelationGetRelid(relation);
+		entry->filterable = false;
+		RelationClose(relation);
+	}
+	else
+	{
+		entry->relid = InvalidOid;
+		entry->filterable = true;
+	}
+
+	rb->can_filter_change = entry->filterable;
+
+	ReorderBufferFreeSnap(rb, snapshot_now);
+
+	TeardownHistoricSnapshot(false);
+
+	/*
+	 * Aborting the current (sub-)transaction as a whole has the right
+	 * semantics. We want all locks acquired in here to be released, not
+	 * reassigned to the parent and we do not want any database access have
+	 * persistent effects.
+	 */
+	AbortCurrentTransaction();
+
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	return entry->filterable;
+}
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index b0f26c7..7d805d4 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -622,6 +622,7 @@ struct ReorderBuffer
 	 * Private memory context.
 	 */
 	MemoryContext context;
+	MemoryContextCallback relfile_callback;
 
 	/*
 	 * Memory contexts for specific types objects
@@ -642,6 +643,12 @@ struct ReorderBuffer
 	/* Max-heap for sizes of all top-level and sub transactions */
 	pairingheap *txn_heap;
 
+	/* should we try to filter the change? */
+	bool		can_filter_change;
+
+	/* number of changes after a failed attempt at filtering */
+	int8		processed_changes;
+
 	/*
 	 * Statistics about transactions spilled to disk.
 	 *
@@ -742,4 +749,9 @@ extern void ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr);
 
 extern void StartupReorderBuffer(void);
 
+extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+												XLogRecPtr lsn, RelFileLocator *rlocator,
+												bool has_tuple);
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
+
 #endif
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 656ecd9..9b43262 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2433,6 +2433,8 @@ ReorderBufferIterTXNEntry
 ReorderBufferIterTXNState
 ReorderBufferMessageCB
 ReorderBufferPrepareCB
+ReorderBufferRelFileLocatorEnt
+ReorderBufferRelFileLocatorKey
 ReorderBufferRollbackPreparedCB
 ReorderBufferStreamAbortCB
 ReorderBufferStreamChangeCB
-- 
1.8.3.1

#17Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#16)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Some review comments for the patch v13-0002.

======
src/backend/replication/logical/reorderbuffer.c

1. GENERAL

I felt that a summary/overview of how all this filter/cache logic
works should be given in the large file header comment at the top of
this file. There may be some overlap with comments that are already in
the "RelFileLocator filtering" section.

~~~

ReorderBufferRelFileLocatorEnt:

2.
+/* entry for hash table we use to track if the relation can be filtered */
+typedef struct ReorderBufferRelFileLocatorEnt

/* Hash table entry used to determine if the relation can be filtered. */

~~~

ReorderBufferQueueChange:

3.
+ /*
+ * If we're not filtering and we've crossed the change threshold,
+ * attempt to filter again
+ */

SUGGESTION
If filtering was suspended, and we've crossed the change threshold
then reenable filtering.

~~~

ReorderBufferGetRelation:

4.
+static Relation
+ReorderBufferGetRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+ bool has_tuple)

Would a better name be ReorderBufferGetRelationForDecoding(). Yeah,
it's a bit long but perhaps it explains the context/purpose better.

~~~

5.
+ if (filterable)
+ {
+ RelationClose(relation);
+ return NULL;
+ }

I wonder if some small descriptive comment would be helpful here just
to say we are returning NULL to indicate that this relation is not
needed and yada yada...

~~~

RelFileLocatorCacheInvalidateCallback:

6.
+ /* slightly inefficient algorithm but not performance critical path */
+ while ((entry = (ReorderBufferRelFileLocatorEnt *)
hash_seq_search(&status)) != NULL)
+ {
+ /*
+ * If relid is InvalidOid, signaling a complete reset, we must remove
+ * all entries, otherwise just remove the specific relation's entry.
+ * Always remove negative cache entries.
+ */
+ if (relid == InvalidOid || /* complete reset */
+ entry->relid == InvalidOid || /* negative cache entry */
+ entry->relid == relid) /* individual flushed relation */

6a.
Maybe uppercase that 1st comment.

~

6b.
It seems a bit unusual to be referring to "negative cache entries". I
thought it should be described in terms of InvalidOid since that is
what it is using in the condition.

~

6c.
If the relid parameter can take special values like "If relid is
InvalidOid, signaling a complete reset" that sounds like the kind of
thing that should be documented in the function comment.

~~~

ReorderBufferFilterByRelFileLocator

7.
Despite the extra indenting required, I wondered if the logic may be
easier to read (e.g. it shows the association of the
rb->can_filter_change and entry->filterable more clearly) if this is
refactored slightly by sharing a single common return like below:

BEFORE
...
+ key.reltablespace = rlocator->spcOid;
+ key.relfilenumber = rlocator->relNumber;
+ entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+ if (found)
+ {
+ rb->can_filter_change = entry->filterable;
+ return entry->filterable;
+ }
...
+ rb->can_filter_change = entry->filterable;
...
+ return entry->filterable;
+}
AFTER
...
+ key.reltablespace = rlocator->spcOid;
+ key.relfilenumber = rlocator->relNumber;
+ entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+ if (!found)
+ {
...
+ }
+
+ rb->can_filter_change = entry->filterable;
+ return entry->filterable;
+}

======
src/include/replication/reorderbuffer.h

8.
+ /* should we try to filter the change? */
+ bool can_filter_change;
+

I think most of my difficulty reading this patch was due to this field
name 'can_filter_change'.

'can_filter_change' somehow sounded to me like it is past tense. e.g.
like as if we already found some change and we yes, we CAN filter it.

But AFAICT the real meaning is simply that (when the flag is true) we
are ALLOWED to check to see if there is anything filterable. In fact,
the change may or may not be filterable.

Can this be renamed to something more "correct"? e.g.
- 'allow_change_filtering'
- 'enable_change_filtering'
- etc.

~~

9.
+ /* number of changes after a failed attempt at filtering */
+ int8 processed_changes;

Maybe 'unfiltered_changes_count' is a better name for this field?

~~~

10.
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);

Should match the 'can_filter_change' field name, so if you change that
(see comment #8), then change this too.

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#18Amit Kapila
amit.kapila16@gmail.com
In reply to: Ajin Cherian (#16)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Wed, Feb 12, 2025 at 10:41 AM Ajin Cherian <itsajin@gmail.com> wrote:

On Wed, Jan 29, 2025 at 9:31 AM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v12-0001.

======
Commit message

1.
Track transactions which have snapshot changes with a new flag
RBTXN_HAS_SNAPSHOT_CHANGES

~

The commit message only says *what* it does, but not *why* this patch
even exists. TBH, I don't understand why this patch needs to be
separated from your patch 0002, because 0001 makes no independent use
of the flag, nor is it separately tested.

Anyway, if it is going to remain separated then IMO at least the the
message should explain the intended purpose e.g. why the subsequent
patches require this flagged info and how they will use it.

Fixed.

I still can't get from 0001's commit message the reason for tracking
the snapshot changes separately. Also, please find my comments for
0002's commit message.

When most changes in a transaction are unfilterable, the overhead of
starting a transaction for each record is significant.

Can you tell what is the exact overhead by testing it? IIRC, that was
the initial approach. It is better if you can mention in the commit
message what was overhead. It will be easier for reviewers.

To reduce this overhead a hash cache of relation file locators is
created. Even then a hash search for every record before recording has
considerable overhead especially for use cases where most tables in an
instance are published.

Again, can you share the link of performance data for this overhead
and if you have not published then please share it and also mention it
in commit message?

To further reduce this overhead a simple approach is used to suspend
filtering for a certain number of changes (100) when an unfilterable
change is encountered. In other words, continue filtering changes if
the last record was filtered out. If an unfilterable change is found,
skip filtering the next 100 changes.

Can we try different thresholds for this like 10, 50, 100, 200, etc.
to decide what is a good threshold value to skip filtering changes?

--
With Regards,
Amit Kapila.

#19Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#16)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin.

FYI - Patch set v13* no longer applies cleanly. Needs rebasing.

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#20Ajin Cherian
itsajin@gmail.com
In reply to: Peter Smith (#19)
2 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Mon, Feb 17, 2025 at 10:08 AM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin.

FYI - Patch set v13* no longer applies cleanly. Needs rebasing.

I've rebased the patch. I've also merged patch 1 into patch 2 as the
functionality of the changes in patch 1 are only n patch 2.

On Mon, Feb 17, 2025 at 10:08 AM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin.

FYI - Patch set v13* no longer applies cleanly. Needs rebasing.

I've rebased the patch. I've also merged patch 1 into patch 2 as the
functionality of the changes in patch 1 are only in patch 2. So only 2
patches in this version.

On Fri, Feb 14, 2025 at 6:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

Again, can you share the link of performance data for this overhead
and if you have not published then please share it and also mention it
in commit message?

I will run the performance tests and post an update with the results.

Can we try different thresholds for this like 10, 50, 100, 200, etc.
to decide what is a good threshold value to skip filtering changes?

Ok, will do this. For this patch, I reset the count to 0, else the
test case fails as filtering could be skipped due to throttling. I
think we need a way for the user to set this threshold via a GUC and
that can be used for testing.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

v14-0001-Filter-transactions-that-need-not-be-published.patchapplication/octet-stream; name=v14-0001-Filter-transactions-that-need-not-be-published.patchDownload
From 56714f125f0ec94760a6964c7935129f498bb888 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Mon, 17 Feb 2025 04:53:15 -0500
Subject: [PATCH v14 1/2] Filter transactions that need not be published.

This adds logic to filter transactions early (at decode time, rather than at streaming time) so
they can be skipped if they do not contain tables that are part of the publication list of the
logical replication walsender.

Determining whether to filter a change requires information about the relation
and the publication from the catalog, requiring a transaction to be started.
When most changes in a transaction are unfilterable, the overhead of starting a
transaction for each record is significant. To reduce this overhead a hash cache of relation file
locators is created. Even then a hash search for every record before recording has considerable
overhead especially for use cases where most tables in an instance are published. To further reduce
this overhead a simple approach is used to suspend filtering for a certain number of changes
(100) when an unfilterable change is encountered. In other words, continue filtering changes if
the last record was filtered out. If an unfilterable change is found, skip filtering the next 100
changes.
---
 src/backend/replication/logical/decode.c        |   5 +
 src/backend/replication/logical/reorderbuffer.c | 453 ++++++++++++++++++++----
 src/include/replication/reorderbuffer.h         |  19 +
 src/tools/pgindent/typedefs.list                |   2 +
 4 files changed, 411 insertions(+), 68 deletions(-)

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 24d88f3..978e38a 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	if (ctx->reorder->can_filter_change &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator, true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	if (!(xlrec->flags & XLH_INSERT_IS_SPECULATIVE))
 		change->action = REORDER_BUFFER_CHANGE_INSERT;
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index b42f400..d2e06ca 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -82,6 +82,22 @@
  *	  a bit more memory to the oldest subtransactions, because it's likely
  *	  they are the source for the next sequence of changes.
  *
+ *	  We also try and filter changes that are not relevant for logical decoding
+ *	  as well as give the option for plugins to filter changes in advance.
+ *	  Determining whether to filter a change requires information about the
+ *	  relation from the catalog, requring a transaction to be started.
+ *	  When most changes in a transaction are unfilterable, the overhead of
+ *    starting a transaction for each record is significant. To reduce this
+ *    overhead a hash cache of relation file locators is created. Even then a
+ *    hash search for every record before recording has considerable overhead
+ *    especially for use cases where most tables in an instance are not filtered.
+ *    To further reduce this overhead a simple approach is used to suspend
+ *    filtering for a certain number of changes CHANGES_THRESHOLD_FOR_FILTER
+ *    when an unfilterable change is encountered. In other words, continue
+ *    filtering changes if the last record was filtered out. If an unfilterable
+ *    change is found, skip filtering the next CHANGES_THRESHOLD_FOR_FILTER
+ *    changes.
+ *
  * -------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -109,6 +125,7 @@
 #include "storage/procarray.h"
 #include "storage/sinval.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 #include "utils/relfilenumbermap.h"
@@ -227,8 +244,10 @@ static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
 											   XLogRecPtr lsn, bool create_as_top);
 static void ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
 											  ReorderBufferTXN *subtxn);
-
 static void AssertTXNLsnOrder(ReorderBuffer *rb);
+static Relation ReorderBufferGetRelation(ReorderBuffer *rb,
+										 RelFileLocator *rlocator,
+										 bool has_tuple);
 
 /* ---------------------------------------
  * support functions for lsn-order iterating over the ->changes of a
@@ -279,6 +298,8 @@ static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
  */
 static inline bool ReorderBufferCanStream(ReorderBuffer *rb);
 static inline bool ReorderBufferCanStartStreaming(ReorderBuffer *rb);
+static Snapshot ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb,
+											   ReorderBufferTXN *txn);
 static void ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
 static void ReorderBufferStreamCommit(ReorderBuffer *rb, ReorderBufferTXN *txn);
 
@@ -305,6 +326,48 @@ static void ReorderBufferChangeMemoryUpdate(ReorderBuffer *rb,
 											bool addition, Size sz);
 
 /*
+ * ---------------------------------------
+ * RelFileLocator filtering
+ * ---------------------------------------
+ * This hash table serves as a lookup table for determining if a relation can
+ * be filtered before being decoded and queued into the buffer.
+ *
+ * The hash table shares the same lifespan as the reorder buffer. This is
+ * crucial because each reorderbuffer may have different configurations or be
+ * associated with different output plugins, affecting the types of tables to
+ * be processed.
+ */
+
+static HTAB *RelFileLocatorFilterCache = NULL;
+
+static bool relation_callbacks_registered = false;
+
+typedef struct ReorderBufferRelFileLocatorKey
+{
+	Oid			reltablespace;
+	RelFileNumber relfilenumber;
+} ReorderBufferRelFileLocatorKey;
+
+/* Hash table entry used to determine if the relation can be filtered. */
+typedef struct ReorderBufferRelFileLocatorEnt
+{
+	ReorderBufferRelFileLocatorKey key;
+	Oid			relid;
+	bool		filterable;
+} ReorderBufferRelFileLocatorEnt;
+
+static void RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid);
+static void ReorderBufferMemoryResetcallback(void *arg);
+
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100
+
+/*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
  * prior ReorderBuffer instances for the same slot.
  */
@@ -362,6 +425,34 @@ ReorderBufferAllocate(void)
 	buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
 								 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
+	/*
+	 * To support early filtering of changes, this hash table acts
+	 * as a lookup table to determine if the corresponding relation is
+	 * required to be decoded and queued into the buffer. The hash table
+	 * shares the same lifespan as the reorder buffer.
+	 *
+	 * Also setup the callback to invalidate cache when relations are updated.
+	 */
+	hash_ctl.keysize = sizeof(ReorderBufferRelFileLocatorKey);
+	hash_ctl.entrysize = sizeof(ReorderBufferRelFileLocatorEnt);
+	hash_ctl.hcxt = buffer->context;
+
+	RelFileLocatorFilterCache =
+		hash_create("RelFileLocatorFilterCache", 1000, &hash_ctl,
+					HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	buffer->relfile_callback.arg = NULL;
+	buffer->relfile_callback.func = ReorderBufferMemoryResetcallback;
+	MemoryContextRegisterResetCallback(buffer->context, &buffer->relfile_callback);
+
+	if (!relation_callbacks_registered)
+	{
+		/* Watch for invalidation events. */
+		CacheRegisterRelcacheCallback(RelFileLocatorCacheInvalidateCallback,
+									  (Datum) 0);
+		relation_callbacks_registered = true;
+	}
+
 	buffer->by_txn_last_xid = InvalidTransactionId;
 	buffer->by_txn_last_txn = NULL;
 
@@ -372,6 +463,8 @@ ReorderBufferAllocate(void)
 	/* txn_heap is ordered by transaction size */
 	buffer->txn_heap = pairingheap_allocate(ReorderBufferTXNSizeCompare, NULL);
 
+	buffer->can_filter_change = true;
+
 	buffer->spillTxns = 0;
 	buffer->spillCount = 0;
 	buffer->spillBytes = 0;
@@ -826,6 +919,14 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 
 		toptxn->txn_flags |= RBTXN_HAS_STREAMABLE_CHANGE;
 	}
+	else if (change->action == REORDER_BUFFER_CHANGE_INVALIDATION ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID)
+	{
+		ReorderBufferTXN *toptxn = rbtxn_get_toptxn(txn);
+
+		toptxn->txn_flags |= RBTXN_HAS_SNAPSHOT_CHANGES;
+	}
 
 	change->lsn = lsn;
 	change->txn = txn;
@@ -835,6 +936,17 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 	txn->nentries++;
 	txn->nentries_mem++;
 
+	/*
+	 * If filtering was suspended and we've crossed the change threshold,
+	 * attempt to filter again
+	 */
+	if (!rb->can_filter_change && (++rb->unfiltered_changes_count
+									>= CHANGES_THRESHOLD_FOR_FILTER))
+	{
+		rb->can_filter_change = true;
+		rb->unfiltered_changes_count = 0;
+	}
+
 	/* update memory accounting information */
 	ReorderBufferChangeMemoryUpdate(rb, change, NULL, true,
 									ReorderBufferChangeSize(change));
@@ -1734,6 +1846,9 @@ ReorderBufferTruncateTXN(ReorderBuffer *rb, ReorderBufferTXN *txn, bool txn_prep
 		txn->txn_flags |= RBTXN_IS_SERIALIZED_CLEAR;
 	}
 
+	/* All snapshot changes up to this point have been processed. */
+	txn->txn_flags &= ~RBTXN_HAS_SNAPSHOT_CHANGES;
+
 	/* also reset the number of entries in the transaction */
 	txn->nentries_mem = 0;
 	txn->nentries = 0;
@@ -2177,6 +2292,82 @@ ReorderBufferResetTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 }
 
 /*
+ * Return the relation corresponding to the given RelFileLocator 'rlocator' if
+ * relevant to decoding. Otherwise, return NULL.
+ *
+ * We don't decode catalog files, relations that are not logically logged,
+ * temporary heaps and sequences.
+ */
+static Relation
+ReorderBufferGetRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+						 bool has_tuple)
+{
+	bool		filterable = false;
+	Relation	relation;
+	Oid			reloid;
+
+	reloid = RelidByRelfilenumber(rlocator->spcOid, rlocator->relNumber);
+
+	if (!OidIsValid(reloid))
+	{
+		if (!has_tuple)
+		{
+			/*
+			 * Mapped catalog tuple without data, emitted while catalog table was in
+			 * the process of being rewritten. We can fail to look up the
+			 * relfilenumber, because the relmapper has no "historic" view, in
+			 * contrast to the normal catalog during decoding. Thus repeated rewrites
+			 * can cause a lookup failure. That's OK because we do not decode catalog
+			 * changes anyway. Normally such tuples would be skipped over below, but
+			 * we can't identify whether the table should be logically logged without
+			 * mapping the relfilenumber to the oid.
+			 */
+			return NULL;
+		}
+
+		elog(ERROR, "could not map filenumber \"%s\" to relation OID",
+				relpathperm(*rlocator, MAIN_FORKNUM));
+	}
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid, relpathperm(*rlocator, MAIN_FORKNUM));
+
+	if (!RelationIsLogicallyLogged(relation))
+		filterable = true;
+
+	else if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		/*
+		 * Ignore temporary heaps created during DDL unless the plugin has asked
+		 * for them.
+		 */
+		filterable = true;
+	}
+
+	else if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		/*
+		 * For now ignore sequence changes entirely. Most of the time they don't
+		 * log changes using records we understand, so it doesn't make sense to
+		 * handle the few cases we do.
+		 */
+		filterable = true;
+	}
+
+	/* Return NULL to indicate that this relation need not be decoded. */
+	if (filterable)
+	{
+		RelationClose(relation);
+		return NULL;
+	}
+
+	return relation;
+}
+
+/*
  * Helper function for ReorderBufferReplay and ReorderBufferStreamTXN.
  *
  * Send data of a transaction (and its subtransactions) to the
@@ -2248,7 +2439,6 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 		while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
 		{
 			Relation	relation = NULL;
-			Oid			reloid;
 
 			CHECK_FOR_INTERRUPTS();
 
@@ -2307,55 +2497,11 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 				case REORDER_BUFFER_CHANGE_DELETE:
 					Assert(snapshot_now);
 
-					reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
-												  change->data.tp.rlocator.relNumber);
-
-					/*
-					 * Mapped catalog tuple without data, emitted while
-					 * catalog table was in the process of being rewritten. We
-					 * can fail to look up the relfilenumber, because the
-					 * relmapper has no "historic" view, in contrast to the
-					 * normal catalog during decoding. Thus repeated rewrites
-					 * can cause a lookup failure. That's OK because we do not
-					 * decode catalog changes anyway. Normally such tuples
-					 * would be skipped over below, but we can't identify
-					 * whether the table should be logically logged without
-					 * mapping the relfilenumber to the oid.
-					 */
-					if (reloid == InvalidOid &&
-						change->data.tp.newtuple == NULL &&
-						change->data.tp.oldtuple == NULL)
-						goto change_done;
-					else if (reloid == InvalidOid)
-						elog(ERROR, "could not map filenumber \"%s\" to relation OID",
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM));
-
-					relation = RelationIdGetRelation(reloid);
+					relation = ReorderBufferGetRelation(rb, &change->data.tp.rlocator,
+														(change->data.tp.newtuple != NULL ||
+														 change->data.tp.oldtuple != NULL));
 
 					if (!RelationIsValid(relation))
-						elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
-							 reloid,
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM));
-
-					if (!RelationIsLogicallyLogged(relation))
-						goto change_done;
-
-					/*
-					 * Ignore temporary heaps created during DDL unless the
-					 * plugin has asked for them.
-					 */
-					if (relation->rd_rel->relrewrite && !rb->output_rewrites)
-						goto change_done;
-
-					/*
-					 * For now ignore sequence changes entirely. Most of the
-					 * time they don't log changes using records we
-					 * understand, so it doesn't make sense to handle the few
-					 * cases we do.
-					 */
-					if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
 						goto change_done;
 
 					/* user-triggered change */
@@ -4171,19 +4317,13 @@ ReorderBufferCanStartStreaming(ReorderBuffer *rb)
 }
 
 /*
- * Send data of a large transaction (and its subtransactions) to the
- * output plugin, but using the stream API.
+ * This function generates or retrieves a consistent snapshot of a transaction
+ * that is currently in progress for logical replication.
  */
-static void
-ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+static Snapshot
+ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb, ReorderBufferTXN *txn)
 {
 	Snapshot	snapshot_now;
-	CommandId	command_id;
-	Size		stream_bytes;
-	bool		txn_is_streamed;
-
-	/* We can never reach here for a subtransaction. */
-	Assert(rbtxn_is_toptxn(txn));
 
 	/*
 	 * We can't make any assumptions about base snapshot here, similar to what
@@ -4226,12 +4366,11 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		if (txn->base_snapshot == NULL)
 		{
 			Assert(txn->ninvalidations == 0);
-			return;
+			return NULL;
 		}
 
-		command_id = FirstCommandId;
 		snapshot_now = ReorderBufferCopySnap(rb, txn->base_snapshot,
-											 txn, command_id);
+											 txn, FirstCommandId);
 	}
 	else
 	{
@@ -4244,17 +4383,40 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		 * the LSN condition in the previous branch (so no need to walk
 		 * through subxacts again). In fact, we must not do that as we may be
 		 * using snapshot half-way through the subxact.
-		 */
-		command_id = txn->command_id;
-
-		/*
+		 *
 		 * We can't use txn->snapshot_now directly because after the last
 		 * streaming run, we might have got some new sub-transactions. So we
 		 * need to add them to the snapshot.
 		 */
 		snapshot_now = ReorderBufferCopySnap(rb, txn->snapshot_now,
-											 txn, command_id);
+											 txn, txn->command_id);
+	}
+
+	return snapshot_now;
+}
+
+/*
+ * Send data of a large transaction (and its subtransactions) to the
+ * output plugin, but using the stream API.
+ */
+static void
+ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+	Snapshot	snapshot_now;
+	Size		stream_bytes;
+	bool		txn_is_streamed;
+
+	/* We can never reach here for a subtransaction. */
+	Assert(rbtxn_is_toptxn(txn));
+
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, txn);
+
+	/* This transaction didn't make any changes to the database till now. */
+	if (snapshot_now == NULL)
+		return;
 
+	if (txn->snapshot_now != NULL)
+	{
 		/* Free the previously copied snapshot. */
 		Assert(txn->snapshot_now->copied);
 		ReorderBufferFreeSnap(rb, txn->snapshot_now);
@@ -4272,7 +4434,7 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 
 	/* Process and send the changes to output plugin. */
 	ReorderBufferProcessTXN(rb, txn, InvalidXLogRecPtr, snapshot_now,
-							command_id, true);
+							snapshot_now->curcid, true);
 
 	rb->streamCount += 1;
 	rb->streamBytes += stream_bytes;
@@ -5461,3 +5623,158 @@ restart:
 		*cmax = ent->cmax;
 	return true;
 }
+
+/*
+ * Flush mapping entries when the corresponding relations are updated in pg_class.
+ * If relid is InvalidOid then do a complete reset of the RelFileLocatorFilterCache.
+ */
+static void
+RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid)
+{
+	HASH_SEQ_STATUS status;
+	ReorderBufferRelFileLocatorEnt *entry;
+
+	if (!RelFileLocatorFilterCache)
+		return;
+
+	hash_seq_init(&status, RelFileLocatorFilterCache);
+
+	/* Slightly inefficient algorithm but not performance critical path */
+	while ((entry = (ReorderBufferRelFileLocatorEnt *) hash_seq_search(&status)) != NULL)
+	{
+		/*
+		 * If relid is InvalidOid, signaling a complete reset, we must remove
+		 * all entries, otherwise just remove the specific relation's entry.
+		 * Always remove negative cache entries.
+		 */
+		if (relid == InvalidOid ||	/* complete reset */
+			entry->relid == InvalidOid ||	/* invalid cache entry */
+			entry->relid == relid)	/* individual flushed relation */
+		{
+			if (hash_search(RelFileLocatorFilterCache,
+							&entry->key,
+							HASH_REMOVE,
+							NULL) == NULL)
+				elog(ERROR, "hash table corrupted");
+		}
+	}
+}
+
+/*
+ * Context reset/delete callback for RelFileLocatorFilterCache.
+ */
+static void
+ReorderBufferMemoryResetcallback(void *arg)
+{
+	RelFileLocatorFilterCache = NULL;
+}
+
+/*
+ * Determine whether the record corresponding to the relation identified by
+ * 'rlocator' needs to be filtered and not decoded and queued in the buffer.
+ *
+ * Determining the necessity of decoding requires accessing relation
+ * information from the system catalog, which requires a historical snapshot.
+ * We cannot directly use the current snapshot of the transaction due to the
+ * presence of INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION records in the
+ * buffer that could modify the snapshot. A proper snapshot can only be
+ * constructed after these records are processed in ReorderBufferProcessTXN, or
+ * if we are decoding a transaction without these records. See comment on top
+ * of ReorderBufferGetRelation() to see list of relations that are not
+ * decoded by the reorderbuffer.
+ *
+ * To reduce the overhead from the system catalog access and transaction
+ * management, the results are cached in the 'RelFileLocatorFilterCache' hash
+ * table.
+ *
+ * Returns true if the relation can be filtered; otherwise, false.
+ */
+bool
+ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+									XLogRecPtr lsn, RelFileLocator *rlocator,
+									bool has_tuple)
+{
+	bool		found;
+	Relation	relation;
+	bool		using_subtxn;
+	Snapshot	snapshot_now;
+	ReorderBufferTXN *txn,
+			   *toptxn;
+	ReorderBufferRelFileLocatorEnt *entry;
+	ReorderBufferRelFileLocatorKey key;
+
+	Assert(RelFileLocatorFilterCache);
+
+	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+	Assert(txn);
+	toptxn = rbtxn_get_toptxn(txn);
+	rb->can_filter_change = false;
+
+	/*
+	 * We cannot construct an accurate historical snapshot until all pending
+	 * records in this transaction that might update the snapshot are
+	 * processed.
+	 */
+	if (rbtxn_has_snapshot_changes(toptxn))
+		return false;
+
+	key.reltablespace = rlocator->spcOid;
+	key.relfilenumber = rlocator->relNumber;
+	entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+	if (found)
+	{
+		rb->can_filter_change = entry->filterable;
+		return entry->filterable;
+	}
+
+	/* constructs a temporary historical snapshot */
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, toptxn);
+	Assert(snapshot_now);
+
+	/* build data to be able to lookup the CommandIds of catalog tuples */
+	ReorderBufferBuildTupleCidHash(rb, toptxn);
+
+	/* setup the initial snapshot */
+	SetupHistoricSnapshot(snapshot_now, toptxn->tuplecid_hash);
+
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by RelFileLocator");
+	else
+		StartTransactionCommand();
+
+	relation = ReorderBufferGetRelation(rb, rlocator, has_tuple);
+
+	if (RelationIsValid(relation))
+	{
+		entry->relid = RelationGetRelid(relation);
+		entry->filterable = false;
+		RelationClose(relation);
+	}
+	else
+	{
+		entry->relid = InvalidOid;
+		entry->filterable = true;
+	}
+
+	rb->can_filter_change = entry->filterable;
+
+	ReorderBufferFreeSnap(rb, snapshot_now);
+
+	TeardownHistoricSnapshot(false);
+
+	/*
+	 * Aborting the current (sub-)transaction as a whole has the right
+	 * semantics. We want all locks acquired in here to be released, not
+	 * reassigned to the parent and we do not want any database access have
+	 * persistent effects.
+	 */
+	AbortCurrentTransaction();
+
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	return entry->filterable;
+}
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 517a8e3..449d8a2 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -176,6 +176,7 @@ typedef struct ReorderBufferChange
 #define RBTXN_SENT_PREPARE			0x0200
 #define RBTXN_IS_COMMITTED			0x0400
 #define RBTXN_IS_ABORTED			0x0800
+#define RBTXN_HAS_SNAPSHOT_CHANGES  0x1000
 
 #define RBTXN_PREPARE_STATUS_MASK	(RBTXN_IS_PREPARED | RBTXN_SKIPPED_PREPARE | RBTXN_SENT_PREPARE)
 
@@ -215,6 +216,12 @@ typedef struct ReorderBufferChange
 	((txn)->txn_flags & RBTXN_HAS_STREAMABLE_CHANGE) != 0 \
 )
 
+/* Does this transaction make changes to the current snapshot? */
+#define rbtxn_has_snapshot_changes(txn) \
+( \
+	((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
+)
+
 /*
  * Has this transaction been streamed to downstream?
  *
@@ -641,6 +648,7 @@ struct ReorderBuffer
 	 * Private memory context.
 	 */
 	MemoryContext context;
+	MemoryContextCallback relfile_callback;
 
 	/*
 	 * Memory contexts for specific types objects
@@ -661,6 +669,12 @@ struct ReorderBuffer
 	/* Max-heap for sizes of all top-level and sub transactions */
 	pairingheap *txn_heap;
 
+	/* should we try to filter the change? */
+	bool		can_filter_change;
+
+	/* number of changes after a failed attempt at filtering */
+	int8		unfiltered_changes_count;
+
 	/*
 	 * Statistics about transactions spilled to disk.
 	 *
@@ -761,4 +775,9 @@ extern void ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr);
 
 extern void StartupReorderBuffer(void);
 
+extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+												XLogRecPtr lsn, RelFileLocator *rlocator,
+												bool has_tuple);
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
+
 #endif
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index b6c170a..2c76247 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2434,6 +2434,8 @@ ReorderBufferIterTXNEntry
 ReorderBufferIterTXNState
 ReorderBufferMessageCB
 ReorderBufferPrepareCB
+ReorderBufferRelFileLocatorEnt
+ReorderBufferRelFileLocatorKey
 ReorderBufferRollbackPreparedCB
 ReorderBufferStreamAbortCB
 ReorderBufferStreamChangeCB
-- 
1.8.3.1

v14-0002-Introduce-a-output-plugin-callback-to-filter-cha.patchapplication/octet-stream; name=v14-0002-Introduce-a-output-plugin-callback-to-filter-cha.patchDownload
From 939a090b1a883f9aaa0672a0f3b88694af320af5 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Mon, 17 Feb 2025 05:29:06 -0500
Subject: [PATCH v14 2/2] Introduce a output plugin callback to filter changes

This new output plugin callback provides an option to logical decoding plugins to filter out
changes early. The primary purpose of the callback is to conserve memory and processing cycles
by excluding changes that are not required by output plugins.
---
 doc/src/sgml/logicaldecoding.sgml               | 41 ++++++++++++++-
 src/backend/replication/logical/decode.c        | 70 ++++++++++++++++++++++++-
 src/backend/replication/logical/logical.c       | 42 +++++++++++++++
 src/backend/replication/logical/reorderbuffer.c | 50 ++++++++++++++----
 src/backend/replication/pgoutput/pgoutput.c     | 70 +++++++++++++++++++++++++
 src/include/replication/output_plugin.h         | 20 +++++++
 src/include/replication/reorderbuffer.h         |  8 +++
 src/test/subscription/t/001_rep_changes.pl      | 18 ++++++-
 src/test/subscription/t/013_partition.pl        |  7 +++
 9 files changed, 311 insertions(+), 15 deletions(-)

diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 1c4ae38..a603c47 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -560,6 +560,7 @@ typedef struct OutputPluginCallbacks
     LogicalDecodeCommitCB commit_cb;
     LogicalDecodeMessageCB message_cb;
     LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+    LogicalDecodeFilterChangeCB filter_change_cb;
     LogicalDecodeShutdownCB shutdown_cb;
     LogicalDecodeFilterPrepareCB filter_prepare_cb;
     LogicalDecodeBeginPrepareCB begin_prepare_cb;
@@ -582,8 +583,8 @@ typedef void (*LogicalOutputPluginInit) (struct OutputPluginCallbacks *cb);
      and <function>commit_cb</function> callbacks are required,
      while <function>startup_cb</function>, <function>truncate_cb</function>,
      <function>message_cb</function>, <function>filter_by_origin_cb</function>,
-     and <function>shutdown_cb</function> are optional.
-     If <function>truncate_cb</function> is not set but a
+     <function>shutdown_cb</function>, and <function>filter_change_cb</function>
+     are optional. If <function>truncate_cb</function> is not set but a
      <command>TRUNCATE</command> is to be decoded, the action will be ignored.
     </para>
 
@@ -871,6 +872,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
      </para>
      </sect3>
 
+     <sect3 id="logicaldecoding-output-plugin-filter-change">
+     <title>Change Filter Callback</title>
+
+     <para>
+      The optional <function>filter_change_cb</function> is called before a
+      change record is decoded to determine whether the change can be filtered
+      out.
+<programlisting>
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+                                             Oid relid,
+                                             ReorderBufferChangeType change_type,
+                                             bool in_txn, bool *cache_valid);
+</programlisting>
+      To indicate that decoding can be skipped for the given change
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.
+      The <parameter>in_txn</parameter> parameter indicates whether the
+      callback is invoked within a transaction block.
+      When <parameter>in_txn</parameter> is false, and if making a decision to filter a change requires being inside a
+      transaction block, such as needing access to the catalog, set
+      <parameter>*cache_valid</parameter> to <literal>false</literal>.
+      This ensures that the callback will be reinvoked once a transaction block
+      starts. If a decision can be made immediately, set
+      <parameter>*cache_valid</parameter> to <literal>true</literal>.
+     </para>
+     <para>
+      The primary purpose of this callback function is to optimize memory usage
+      and processing efficiency by filtering out changes that are unnecessary for
+      output plugins. It enables output plugins to selectively process relevant
+      changes. Caching filtering decisions locally is recommended, as it enables
+      the callback to provide cached results without repeatedly initiating
+      transactions or querying the catalog. This approach minimizes overhead
+      and improves efficiency during the decoding process.
+     </para>
+     </sect3>
+
     <sect3 id="logicaldecoding-output-plugin-message">
      <title>Generic Message Callback</title>
 
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 978e38a..6a3388e 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -576,6 +576,17 @@ FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
 }
 
 /*
+ * Check if filtering changes before decoding is supported and we're not suppressing filter
+ * changes currently.
+ */
+static inline bool
+FilterChangeIsEnabled(LogicalDecodingContext *ctx)
+{
+	return (ctx->callbacks.filter_change_cb != NULL &&
+				ctx->reorder->can_filter_change);
+}
+
+/*
  * Handle rmgr LOGICALMSG_ID records for LogicalDecodingProcessRecord().
  */
 void
@@ -915,9 +926,16 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
-	if (ctx->reorder->can_filter_change &&
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
 		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
-											buf->origptr, &target_locator, true))
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_INSERT,
+											true))
 		return;
 
 	change = ReorderBufferGetChange(ctx->reorder);
@@ -970,6 +988,18 @@ DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_UPDATE,
+											true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_UPDATE;
 	change->origin_id = XLogRecGetOrigin(r);
@@ -1036,6 +1066,18 @@ DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_DELETE,
+											true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 
 	if (xlrec->flags & XLH_DELETE_IS_SUPER)
@@ -1139,6 +1181,18 @@ DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 		return;
 
 	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &rlocator,
+											REORDER_BUFFER_CHANGE_INSERT,
+											true))
+		return;
+
+	/*
 	 * We know that this multi_insert isn't for a catalog, so the block should
 	 * always have data even if a full-page write of it is taken.
 	 */
@@ -1229,6 +1283,18 @@ DecodeSpecConfirm(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (FilterChangeIsEnabled(ctx) &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator,
+											REORDER_BUFFER_CHANGE_INSERT,
+											true))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM;
 	change->origin_id = XLogRecGetOrigin(r);
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 8ea846b..635e1de 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -74,6 +74,9 @@ static void truncate_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 							   XLogRecPtr message_lsn, bool transactional,
 							   const char *prefix, Size message_size, const char *message);
+static bool filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+								  ReorderBufferChangeType change_type, bool in_txn,
+								  bool *cache_valid);
 
 /* streaming callbacks */
 static void stream_start_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
@@ -220,6 +223,7 @@ StartupDecodingContext(List *output_plugin_options,
 	/* wrap output plugin callbacks, so we can add error context information */
 	ctx->reorder->begin = begin_cb_wrapper;
 	ctx->reorder->apply_change = change_cb_wrapper;
+	ctx->reorder->filter_change = filter_change_cb_wrapper;
 	ctx->reorder->apply_truncate = truncate_cb_wrapper;
 	ctx->reorder->commit = commit_cb_wrapper;
 	ctx->reorder->message = message_cb_wrapper;
@@ -1224,6 +1228,44 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+static bool
+filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+					  ReorderBufferChangeType change_type, bool in_txn,
+					  bool *cache_valid)
+{
+	LogicalDecodingContext *ctx = cache->private_data;
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	/* check if the filter change callback is supported */
+	if (ctx->callbacks.filter_change_cb == NULL)
+		return false;
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_change";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_change_cb(ctx, relid, change_type, in_txn, cache_valid);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index d2e06ca..00046a6 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -361,11 +361,12 @@ static void ReorderBufferMemoryResetcallback(void *arg);
 
 /*
  * After encountering a change that cannot be filtered out, filtering is
- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
  * This strategy helps to minimize the overhead of performing a hash table
  * search for each record, especially when most changes are not filterable.
  */
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0
 
 /*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
@@ -5692,9 +5693,11 @@ ReorderBufferMemoryResetcallback(void *arg)
 bool
 ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 									XLogRecPtr lsn, RelFileLocator *rlocator,
+									ReorderBufferChangeType change_type,
 									bool has_tuple)
 {
 	bool		found;
+	bool		cache_valid;
 	Relation	relation;
 	bool		using_subtxn;
 	Snapshot	snapshot_now;
@@ -5708,7 +5711,6 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
 	Assert(txn);
 	toptxn = rbtxn_get_toptxn(txn);
-	rb->can_filter_change = false;
 
 	/*
 	 * We cannot construct an accurate historical snapshot until all pending
@@ -5725,7 +5727,25 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (found)
 	{
 		rb->can_filter_change = entry->filterable;
-		return entry->filterable;
+
+		/*
+		 * Quick return if we already know that the relation is not to be decoded.
+		 * These are for special relations that are unlogged and for sequences
+		 * and catalogs.
+		 */
+		if (entry->filterable)
+			return true;
+
+		/* Allow the output plugin to filter relations */
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  false, &cache_valid);
+
+		/*
+		 * If plugin had the relation ready in cache, the response is valid, else
+		 * we'll need to call the plugin a second time within a transaction.
+		 */
+		if (cache_valid)
+			return rb->can_filter_change;
 	}
 
 	/* constructs a temporary historical snapshot */
@@ -5749,18 +5769,30 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 
 	if (RelationIsValid(relation))
 	{
-		entry->relid = RelationGetRelid(relation);
+		if (IsToastRelation(relation))
+		{
+			Oid     real_reloid = InvalidOid;
+			char   *toast_name = RelationGetRelationName(relation);
+			/* pg_toast_ len is 9 */
+			char   *start_ch = &toast_name[9];
+
+			real_reloid = pg_strtoint32(start_ch);
+			entry->relid = real_reloid;
+		}
+		else
+			entry->relid = RelationGetRelid(relation);
+
 		entry->filterable = false;
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  true, &cache_valid);
 		RelationClose(relation);
 	}
 	else
 	{
 		entry->relid = InvalidOid;
-		entry->filterable = true;
+		rb->can_filter_change = entry->filterable = true;
 	}
 
-	rb->can_filter_change = entry->filterable;
-
 	ReorderBufferFreeSnap(rb, snapshot_now);
 
 	TeardownHistoricSnapshot(false);
@@ -5776,5 +5808,5 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (using_subtxn)
 		RollbackAndReleaseCurrentSubTransaction();
 
-	return entry->filterable;
+	return rb->can_filter_change;
 }
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index 7d464f6..93eb579 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -57,6 +57,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+								   ReorderBufferChangeType change_type, bool in_txn,
+								   bool *cache_valid);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -268,6 +271,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_change_cb = pgoutput_filter_change;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1744,6 +1748,72 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 }
 
 /*
+ * Determine whether a change to the specified relation should be published.
+ *
+ * See the comments atop of LogicalDecodeFilterChangeCB for details.
+ */
+static bool
+pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+					   ReorderBufferChangeType change_type, bool in_txn, bool *cache_valid)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *entry;
+
+	Assert(RelationSyncCache != NULL);
+
+	if (in_txn)
+	{
+		Relation	relation;
+
+		relation = RelationIdGetRelation(relid);
+		entry = get_rel_sync_entry(data, relation);
+		*cache_valid = true;
+	}
+	else
+	{
+		entry = (RelationSyncEntry *) hash_search(RelationSyncCache,
+												  &relid,
+												  HASH_FIND, cache_valid);
+		if (!*cache_valid)
+			return false;
+	}
+
+	/*
+	 * If the pubaction is not supported by this publication then return true to say
+	 * the change for this entry can be skipped.
+	 */
+	switch (change_type)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			if (!entry->pubactions.pubinsert)
+			{
+				elog(DEBUG1, "Filtering INSERT");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			if (!entry->pubactions.pubupdate)
+			{
+				elog(DEBUG1, "Filtering UPDATE");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			if (!entry->pubactions.pubdelete)
+			{
+				elog(DEBUG1, "Filtering DELETE");
+				return true;
+			}
+			break;
+		default:
+			/* allow any other changes that are not explicitly filtered */
+			return false;
+	}
+
+	return false;
+}
+
+/*
  * Shutdown the output plugin.
  *
  * Note, we don't need to clean the data->context, data->cachectx, and
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 8d4d5b71..f82bc1d 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -97,6 +97,25 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
 											   RepOriginId origin_id);
 
 /*
+ * This callback is called before a change record is decoded to determine
+ * whether the change can be filtered out before entering the reorder buffer.
+ *
+ * Typically, the output plugin needs to refer to the system catalog to make
+ * this decision. To enhance efficiency, the plugin should cache the lookup
+ * result for each relation, minimizing catalog access on subsequent calls.
+ *
+ * If the callback is called with 'in_txn' set as false (the reorder
+ * buffer has not started a transaction), the output plugin should set '*cache_valid'
+ * to false to indicate that the result is not available in its internal cache.
+ * If 'in_txn' is true, the plugin can create a cache entry after querying the
+ * catalog.
+ */
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
+/*
  * Called to shutdown an output plugin.
  */
 typedef void (*LogicalDecodeShutdownCB) (struct LogicalDecodingContext *ctx);
@@ -222,6 +241,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterChangeCB filter_change_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 449d8a2..579a24a 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -494,6 +494,12 @@ typedef void (*ReorderBufferMessageCB) (ReorderBuffer *rb,
 										const char *prefix, Size sz,
 										const char *message);
 
+/* filter change callback signature */
+typedef bool (*ReorderBufferFilterChangeCB) (ReorderBuffer *rb,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
 /* begin prepare callback signature */
 typedef void (*ReorderBufferBeginPrepareCB) (ReorderBuffer *rb,
 											 ReorderBufferTXN *txn);
@@ -604,6 +610,7 @@ struct ReorderBuffer
 	 */
 	ReorderBufferBeginCB begin;
 	ReorderBufferApplyChangeCB apply_change;
+	ReorderBufferFilterChangeCB filter_change;
 	ReorderBufferApplyTruncateCB apply_truncate;
 	ReorderBufferCommitCB commit;
 	ReorderBufferMessageCB message;
@@ -777,6 +784,7 @@ extern void StartupReorderBuffer(void);
 
 extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 												XLogRecPtr lsn, RelFileLocator *rlocator,
+												ReorderBufferChangeType change_type,
 												bool has_tuple);
 extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
 
diff --git a/src/test/subscription/t/001_rep_changes.pl b/src/test/subscription/t/001_rep_changes.pl
index 8726fe0..bced62c 100644
--- a/src/test/subscription/t/001_rep_changes.pl
+++ b/src/test/subscription/t/001_rep_changes.pl
@@ -485,9 +485,10 @@ $node_publisher->wait_for_catchup('tap_sub');
 
 # Check that we don't send BEGIN and COMMIT because of empty transaction
 # optimization.  We have to look for the DEBUG1 log messages about that, so
-# temporarily bump up the log verbosity.
+# temporarily bump up the log verbosity. Also confirm that unpublished
+# changes are filtered out after a restart.
 $node_publisher->append_conf('postgresql.conf', "log_min_messages = debug1");
-$node_publisher->reload;
+$node_publisher->restart;
 
 # Note that the current location of the log file is not grabbed immediately
 # after reloading the configuration, but after sending one SQL command to
@@ -495,6 +496,8 @@ $node_publisher->reload;
 $log_location = -s $node_publisher->logfile;
 
 $node_publisher->safe_psql('postgres', "INSERT INTO tab_notrep VALUES (11)");
+$node_publisher->safe_psql('postgres', "UPDATE tab_notrep SET a = 2 WHERE a = 1");
+$node_publisher->safe_psql('postgres', "DELETE FROM tab_notrep WHERE a = 2");
 
 $node_publisher->wait_for_catchup('tap_sub');
 
@@ -502,6 +505,17 @@ $logfile = slurp_file($node_publisher->logfile, $log_location);
 ok($logfile =~ qr/skipped replication of an empty transaction with XID/,
 	'empty transaction is skipped');
 
+# Check that an unpublished change is filtered out.
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+
+ok($logfile =~ qr/Filtering UPDATE/,
+	'unpublished UPDATE is filtered');
+
+ok($logfile =~ qr/Filtering DELETE/,
+	'unpublished DELETE is filtered');
+
 $result =
   $node_subscriber->safe_psql('postgres', "SELECT count(*) FROM tab_notrep");
 is($result, qq(0), 'check non-replicated table is empty on subscriber');
diff --git a/src/test/subscription/t/013_partition.pl b/src/test/subscription/t/013_partition.pl
index 14a3bea..180eaa0 100644
--- a/src/test/subscription/t/013_partition.pl
+++ b/src/test/subscription/t/013_partition.pl
@@ -468,6 +468,13 @@ $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4 (a int PRIMARY KEY)");
 $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4_1 (a int PRIMARY KEY)");
+
+# Ensure that the subscription 'sub2' catches up with the latest changes. This
+# is necessary for the walsender to update its historic snapshot. Otherwise,
+# the walsender might retain an outdated snapshot, potentially preventing it
+# from accessing the newly created publication.
+$node_publisher->wait_for_catchup('sub2');
+
 # Since we specified publish_via_partition_root in pub_all and
 # pub_lower_level, all partition tables use their root tables' identity and
 # schema. We set the list of publications so that the FOR ALL TABLES
-- 
1.8.3.1

#21Ajin Cherian
itsajin@gmail.com
In reply to: Amit Kapila (#18)
1 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Fri, Feb 14, 2025 at 6:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Feb 12, 2025 at 10:41 AM Ajin Cherian <itsajin@gmail.com> wrote:

On Wed, Jan 29, 2025 at 9:31 AM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v12-0001.

======
Commit message

1.
Track transactions which have snapshot changes with a new flag
RBTXN_HAS_SNAPSHOT_CHANGES

~

The commit message only says *what* it does, but not *why* this patch
even exists. TBH, I don't understand why this patch needs to be
separated from your patch 0002, because 0001 makes no independent use
of the flag, nor is it separately tested.

Anyway, if it is going to remain separated then IMO at least the the
message should explain the intended purpose e.g. why the subsequent
patches require this flagged info and how they will use it.

Fixed.

I still can't get from 0001's commit message the reason for tracking
the snapshot changes separately. Also, please find my comments for
0002's commit message.

When most changes in a transaction are unfilterable, the overhead of
starting a transaction for each record is significant.

Can you tell what is the exact overhead by testing it? IIRC, that was
the initial approach. It is better if you can mention in the commit
message what was overhead. It will be easier for reviewers.

To reduce this overhead a hash cache of relation file locators is
created. Even then a hash search for every record before recording has
considerable overhead especially for use cases where most tables in an
instance are published.

Again, can you share the link of performance data for this overhead
and if you have not published then please share it and also mention it
in commit message?

I compared the patch 1 which does not employ a hash cache and has the
overhead of starting a transaction every time the filter is checked.

I created a test setup of 10 million inserts in 3 different scenarios:
1. All inserts on unpublished tables
2. Half of the inserts on unpublished table and half on pupblished table
3. All inserts on published tables.

The percentage improvement in the new optimized patch compared to the
old patch is:

No transactions in publication: 85.39% improvement
Half transactions in publication: 72.70% improvement
All transactions in publication: 48.47% improvement

Attaching a graph to show the difference.

To further reduce this overhead a simple approach is used to suspend
filtering for a certain number of changes (100) when an unfilterable
change is encountered. In other words, continue filtering changes if
the last record was filtered out. If an unfilterable change is found,
skip filtering the next 100 changes.

Can we try different thresholds for this like 10, 50, 100, 200, etc.
to decide what is a good threshold value to skip filtering changes?

Of Course this performance might vary from setup to setup but I tried
the above setup to compare the 4 different threshold levels

Conclusions:
Lower throttling values yield better performance, particularly when
transactions are included in the publication.
Increasing the throttle limit to 200 transactions causes significant
performance degradation, particularly when half or all transactions
are included.
For optimal performance, a moderate throttling value (100
transactions) may be the best balance between performance and
processing efficiency.

Attaching the graph to show the difference. I'm also attaching the
test script that I used.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

Performance.zipapplication/zip; name=Performance.zipDownload
#22Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#21)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Thu, Feb 20, 2025 at 1:30 PM Ajin Cherian <itsajin@gmail.com> wrote:

On Fri, Feb 14, 2025 at 6:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Wed, Feb 12, 2025 at 10:41 AM Ajin Cherian <itsajin@gmail.com> wrote:

On Wed, Jan 29, 2025 at 9:31 AM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v12-0001.

======
Commit message

1.
Track transactions which have snapshot changes with a new flag
RBTXN_HAS_SNAPSHOT_CHANGES

~

The commit message only says *what* it does, but not *why* this patch
even exists. TBH, I don't understand why this patch needs to be
separated from your patch 0002, because 0001 makes no independent use
of the flag, nor is it separately tested.

Anyway, if it is going to remain separated then IMO at least the the
message should explain the intended purpose e.g. why the subsequent
patches require this flagged info and how they will use it.

Fixed.

I still can't get from 0001's commit message the reason for tracking
the snapshot changes separately. Also, please find my comments for
0002's commit message.

When most changes in a transaction are unfilterable, the overhead of
starting a transaction for each record is significant.

Can you tell what is the exact overhead by testing it? IIRC, that was
the initial approach. It is better if you can mention in the commit
message what was overhead. It will be easier for reviewers.

To reduce this overhead a hash cache of relation file locators is
created. Even then a hash search for every record before recording has
considerable overhead especially for use cases where most tables in an
instance are published.

Again, can you share the link of performance data for this overhead
and if you have not published then please share it and also mention it
in commit message?

I compared the patch 1 which does not employ a hash cache and has the
overhead of starting a transaction every time the filter is checked.

Just to clarify, by patch 1, I mean the first patch in this thread
(v1) posted by Lie Jie here - (1)

1 - /messages/by-id/CAGfChW5Qo2SrjJ7rU9YYtZbRaWv6v-Z8MJn=dQNx4uCSqDEOHA@mail.gmail.com

regards,
Ajin Cherian
Fujitsu Australia

#23Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Ajin Cherian (#21)
RE: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Dear Ajin,

I compared the patch 1 which does not employ a hash cache and has the
overhead of starting a transaction every time the filter is checked.

I created a test setup of 10 million inserts in 3 different scenarios:
1. All inserts on unpublished tables
2. Half of the inserts on unpublished table and half on pupblished table
3. All inserts on published tables.

The percentage improvement in the new optimized patch compared to the
old patch is:

No transactions in publication: 85.39% improvement
Half transactions in publication: 72.70% improvement
All transactions in publication: 48.47% improvement

Attaching a graph to show the difference.

I could not find any comparisons with HEAD. Can you clarify the throughput/latency/memory
usage with HEAD?

Best regards,
Hayato Kuroda
FUJITSU LIMITED

#24Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Ajin Cherian (#20)
RE: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Dear Ajin,

Here are my comments. I must play with patches to understand more.

01.
```
extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
XLogRecPtr lsn, RelFileLocator *rlocator,
ReorderBufferChangeType change_type,
bool has_tuple);
```

Can you explain why "has_tuple is needed? All callers is set to true.

02.
```
static Relation
ReorderBufferGetRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
bool has_tuple)
```

Hmm, the naming is bit confusing for me. This operation is mostly not related with
the reorder buffer. How about "GetPossibleDecodableRelation" or something?

03.
```
if (IsToastRelation(relation))
{
Oid real_reloid = InvalidOid;
char *toast_name = RelationGetRelationName(relation);
/* pg_toast_ len is 9 */
char *start_ch = &toast_name[9];

real_reloid = pg_strtoint32(start_ch);
entry->relid = real_reloid;
}
```

It is bit hacky for me. How about using sscanf like attached?

04.

IIUC, toast tables always require the filter_change() call twice, is it right?
I understood like below:

1. ReorderBufferFilterByRelFileLocator() tries to filter the change at outside the
transaction. The OID indicates the pg_toast_xxx table.
2. pgoutput_filter_change() cannot find the table from the hash. It returns false
with cache_valid=false.
3. ReorderBufferFilterByRelFileLocator() starts a transaction and get its relation.
4. The function recognizes the relation seems toast and get parent oid.
5. The function tries to filter the change in the transaction, with the parent oid.
6. pgoutput_filter_change()->get_rel_sync_entry() enters the parent relation to the
hash and return determine the filtable or not.
7. After sometime, the same table is modified. But the toast table is not stored in
the hash so that whole 1-6 steps are required.

I feel this may affect the perfomance when many toast is modified. How about skiping
the check for toasted ones? ISTM IsToastNamespace() can be used for the decision.

Best regards,
Hayato Kuroda
FUJITSU LIMITED

#25Shlok Kyal
shlok.kyal.oss@gmail.com
In reply to: Ajin Cherian (#20)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Mon, 17 Feb 2025 at 16:20, Ajin Cherian <itsajin@gmail.com> wrote:

On Mon, Feb 17, 2025 at 10:08 AM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin.

FYI - Patch set v13* no longer applies cleanly. Needs rebasing.

I've rebased the patch. I've also merged patch 1 into patch 2 as the
functionality of the changes in patch 1 are only n patch 2.

On Mon, Feb 17, 2025 at 10:08 AM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin.

FYI - Patch set v13* no longer applies cleanly. Needs rebasing.

I've rebased the patch. I've also merged patch 1 into patch 2 as the
functionality of the changes in patch 1 are only in patch 2. So only 2
patches in this version.

On Fri, Feb 14, 2025 at 6:18 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

Again, can you share the link of performance data for this overhead
and if you have not published then please share it and also mention it
in commit message?

I will run the performance tests and post an update with the results.

Can we try different thresholds for this like 10, 50, 100, 200, etc.
to decide what is a good threshold value to skip filtering changes?

Ok, will do this. For this patch, I reset the count to 0, else the
test case fails as filtering could be skipped due to throttling. I
think we need a way for the user to set this threshold via a GUC and
that can be used for testing.

Hi Ajin,

I have reviewed the v14-0001 patch, I have following comment:

1. We need to initialize the 'rb->unfiltered_changes_count = 0', initially
inside function 'ReorderBufferAllocate', otherwise it will get set to garbage
value. While debugging I found out it was setting to 128 by default, so
'++rb->unfiltered_changes_count' was setting it to '-127'. As a result it
might not enter the if condition here for some initial inserts:

+ /*
+ * If filtering was suspended and we've crossed the change threshold,
+ * attempt to filter again
+ */
+ if (!rb->can_filter_change && (++rb->unfiltered_changes_count
+ >= CHANGES_THRESHOLD_FOR_FILTER))
+ {
+ rb->can_filter_change = true;
+ rb->unfiltered_changes_count = 0;
+ }

2. After applying only 0001 patch, then I tried to insert in a published table.
I am getting following error:

postgres=# insert into s1.t1 values(15);
INSERT 0 1
postgres=# insert into s1.t1 values(15);
WARNING: terminating connection because of crash of another server process
DETAIL: The postmaster has commanded this server process to roll back
the current transaction and exit, because another server process
exited abnormally and possibly corrupted shared memory.
HINT: In a moment you should be able to reconnect to the database and
repeat your command.
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
postgres=?#

Stack trace:
#0 __pthread_kill_implementation (no_tid=0, signo=6,
threadid=132913961858944) at ./nptl/pthread_kill.c:44
#1 __pthread_kill_internal (signo=6, threadid=132913961858944) at
./nptl/pthread_kill.c:78
#2 __GI___pthread_kill (threadid=132913961858944,
signo=signo@entry=6) at ./nptl/pthread_kill.c:89
#3 0x000078e270c42476 in __GI_raise (sig=sig@entry=6) at
../sysdeps/posix/raise.c:26
#4 0x000078e270c287f3 in __GI_abort () at ./stdlib/abort.c:79
#5 0x00005c9c56842f53 in ExceptionalCondition
(conditionName=0x5c9c56c10ed5 "MemoryContextIsValid(context)",
fileName=0x5c9c56c10d5f "mcxt.c",
lineNumber=1323) at assert.c:66
#6 0x00005c9c5690e99c in palloc (size=48) at mcxt.c:1323
#7 0x00005c9c55277564 in systable_beginscan
(heapRelation=0x78e2716561d8, indexId=2685, indexOK=true,
snapshot=0x0, nkeys=1, key=0x7ffd9a4446b0)
at genam.c:403
#8 0x00005c9c567cb6ca in SearchCatCacheMiss (cache=0x5c9c58af0e80,
nkeys=1, hashValue=1393055558, hashIndex=6, v1=16384, v2=0, v3=0,
v4=0)
at catcache.c:1533
#9 0x00005c9c567cb43d in SearchCatCacheInternal
(cache=0x5c9c58af0e80, nkeys=1, v1=16384, v2=0, v3=0, v4=0) at
catcache.c:1464
#10 0x00005c9c567cabd8 in SearchCatCache1 (cache=0x5c9c58af0e80,
v1=16384) at catcache.c:1332
#11 0x00005c9c5682a055 in SearchSysCache1 (cacheId=38, key1=16384) at
syscache.c:228
#12 0x00005c9c567e3b6f in get_namespace_name (nspid=16384) at lsyscache.c:3397
#13 0x00005c9c5608eb5e in logicalrep_write_namespace
(out=0x5c9c58b5d5c8, nspid=16384) at proto.c:1033
#14 0x00005c9c5608d0c2 in logicalrep_write_rel (out=0x5c9c58b5d5c8,
xid=0, rel=0x78e27167ede0, columns=0x0,
include_gencols_type=PUBLISH_GENCOLS_NONE) at proto.c:683
#15 0x000078e270fc9301 in send_relation_and_attrs
(relation=0x78e27167ede0, xid=0, ctx=0x5c9c58b4d8f0,
relentry=0x5c9c58b92278) at pgoutput.c:798
#16 0x000078e270fc8ed9 in maybe_send_schema (ctx=0x5c9c58b4d8f0,
change=0x5c9c58b8f2e0, relation=0x78e27167ede0,
relentry=0x5c9c58b92278)
at pgoutput.c:752
#17 0x000078e270fce338 in pgoutput_change (ctx=0x5c9c58b4d8f0,
txn=0x5c9c58b8b2c0, relation=0x78e27167ede0, change=0x5c9c58b8f2e0)
at pgoutput.c:1572
#18 0x00005c9c5607b265 in change_cb_wrapper (cache=0x5c9c58b4f900,
txn=0x5c9c58b8b2c0, relation=0x78e27167ede0, change=0x5c9c58b8f2e0)
at logical.c:1116
#19 0x00005c9c560a2cff in ReorderBufferApplyChange (rb=0x5c9c58b4f900,
txn=0x5c9c58b8b2c0, relation=0x78e27167ede0, change=0x5c9c58b8f2e0,
streaming=false) at reorderbuffer.c:2175
#20 0x00005c9c560a4856 in ReorderBufferProcessTXN (rb=0x5c9c58b4f900,
txn=0x5c9c58b8b2c0, commit_lsn=8380824, snapshot_now=0x5c9c58b736b8,
command_id=0, streaming=false) at reorderbuffer.c:2511
#21 0x00005c9c560a6aef in ReorderBufferReplay (txn=0x5c9c58b8b2c0,
rb=0x5c9c58b4f900, xid=752, commit_lsn=8380824, end_lsn=8380872,
commit_time=793368288662227, origin_id=0, origin_lsn=0) at
reorderbuffer.c:2973
#22 0x00005c9c560a6b71 in ReorderBufferCommit (rb=0x5c9c58b4f900,
xid=752, commit_lsn=8380824, end_lsn=8380872,
commit_time=793368288662227,
origin_id=0, origin_lsn=0) at reorderbuffer.c:2997
#23 0x00005c9c5606804a in DecodeCommit (ctx=0x5c9c58b4d8f0,
buf=0x7ffd9a445010, parsed=0x7ffd9a444e90, xid=752, two_phase=false)
at decode.c:730
#24 0x00005c9c56064d04 in xact_decode (ctx=0x5c9c58b4d8f0,
buf=0x7ffd9a445010) at decode.c:242
#25 0x00005c9c56063fe2 in LogicalDecodingProcessRecord
(ctx=0x5c9c58b4d8f0, record=0x5c9c58b4dd18) at decode.c:116
#26 0x00005c9c56135980 in XLogSendLogical () at walsender.c:3382
#27 0x00005c9c56133895 in WalSndLoop (send_data=0x5c9c56135836
<XLogSendLogical>) at walsender.c:2788
#28 0x00005c9c56130762 in StartLogicalReplication (cmd=0x5c9c58ab9a10)
at walsender.c:1496
#29 0x00005c9c56131ec3 in exec_replication_command (
cmd_string=0x5c9c58a83b90 "START_REPLICATION SLOT \"test1\"
LOGICAL 0/0 (proto_version '4', streaming 'parallel', origin 'any',
publication_names '\"pub1\"')") at walsender.c:2119
#30 0x00005c9c562abbe8 in PostgresMain (dbname=0x5c9c58abd598
"postgres", username=0x5c9c58abd580 "ubuntu") at postgres.c:4687

Thanks and Regards,
Shlok Kyal

#26Ajin Cherian
itsajin@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#23)
1 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Thu, Feb 20, 2025 at 3:08 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:

Dear Ajin,

I compared the patch 1 which does not employ a hash cache and has the
overhead of starting a transaction every time the filter is checked.

I created a test setup of 10 million inserts in 3 different scenarios:
1. All inserts on unpublished tables
2. Half of the inserts on unpublished table and half on pupblished table
3. All inserts on published tables.

The percentage improvement in the new optimized patch compared to the
old patch is:

No transactions in publication: 85.39% improvement
Half transactions in publication: 72.70% improvement
All transactions in publication: 48.47% improvement

Attaching a graph to show the difference.

I could not find any comparisons with HEAD. Can you clarify the throughput/latency/memory
usage with HEAD?

Here's the difference in latency with head. Again 10 million inserts
in 3 scenarios: All transactions on unpublished tables, half of the
transactions on unpublished tables and all transactions on published
tables
Conclusion:
The patched code with 100 transaction throttling significantly
improves performance, reducing execution time by ~69% when no
published transactions are involved, ~43% with partial published
transactions, and ~15% in all published transactions.
Attaching a graph showing the performance differences.

I will run tests comparing memory and throughput as well.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

patched-code-vs-head.pngimage/png; name=patched-code-vs-head.pngDownload
�PNG


IHDRs�����sRGB���gAMA���a	pHYs���o�d��IDATx^��|����xR��BZZ\
�����)����\.������.��p����i?�<���=�Tr��m�~�M������3��m^�aB!�B!�+���B!�B!�#��	!�B!�|��9!�B!���2'�B!��!R��B!�b>D��B!�B��H�B!�B��)sB!�B1"eN!�B!�C��	!�B!�|��9!�B!���2'�B!��!R��B!�b>D��B!�B��H��J*�j�����-EK�-����T�I<[Sz4�kz)���+y.��s8!���://o����������N<�b�x�y9^��Hg�{!^����>�=��_�]�gcV~��?�~�O�~���L���!���J��yGp�)M�#|����������zWx>��m&����Iw����� @���c����<�\�;�cN�mz��]~~�������x��G�\+�|������_n���R[r�%m�u���[,k��}��gv��W����}E�y���\������/���^{�5����l�����}{�?��������VXX����,q������l���7��w�a�m�����j�m=#F��n���[}��}�����|��C���zk/g��9�����YII�}����]��n����{�~��������:w������.kK,��ls��1c���������������v�j+���/[��M ����?��c��]v�e���t^}�U{��g�_f�����"M���?�����_l��i6h������������O��7��E������.������Z*��;��?�`����}���6i�$+++�����k��!C�m������rx�=��SO=ew�}w�tB���;���}��[7�d�����&N�h����m��VK[���G����{� ����z�������|�>��c���+�l;������;c~��w�y�Ks�-�<������1c|]��r��T>h�=�P����m���l���_�������N���;�&L�`���}�o	�����2�,���iI]B=o>��;��|���l~���Mi���6�:�?���
0���-��Y�~��'����l�
7��6��7`�<���X�������C'�h���Z�Q���OtW4
�
(�*�3�<���w_����K/��w8h�^{m[q�����F��7��?��3�����'����>�=:��~����"_�B����<G�z��|�gRQ���}���
%�?����:����>_>�w�i�Q����}g��_���������v��F�[��o���;��>�C��s��4���������[n�%��� �;����K�l�����F�.��r�����[o���Z�Lz�!�����2�`�F:(�����{��8���|G�8��|����;���lp�lC9�Mp�{��
��%�:�v����K���
�����j�F��)=t�v�m�M�6�Mr���o�<��$��{<O��[��m.��Q��������E/���������h?]tQ<x�HD�����V���o��=����=�X7n��TRY���"f�*++�V��!��������[:�}������&�gp3e��F�����7�K���������!�)��^����Cp�����O��l�_���5���L�����+���W��]w���Eg���SN��N;m�<�	�O�@���,�=\�����`aN��8���8��p�H_�u�]�lD �!�����������;�2�5:t���_�A��E�"�x���l�-��3���f
�N9�f�<�H����l�����o��wZg��O>io���t�A�������m��6���M6���C��2E^Q�2�	� S�#<�v�*��p/=O!<��;������X�@Zp1�r��������e.S���!��c���iH�p/�9��t?��\��$���&�uN@�a�������#���
������JI�w�J��A���L��+�#���~�
d�5�h(�������u�Y��cGo���_��29Wf��8�?��lmx?�-����d�A�?��w�3�,T�y�
�"-��QvxO���>q�Y�m�pC7��r��d*W���+w���{�2�y6��p������3���T�7\!l��uK�\�����b�UB�p����{?���O�����y�c�~�����LO�t��{�m���Jd3s���;��^���@y�`�
���+�b�f�X�%��B���J��1D�������KUl������m���~� �$Ku���9�����O?�g4�!`��)���p2CA���n**�����C����>�GN	�N;��+c��|��q����}�������sB�OG��F��2���fS�q#;KaB��0R92��������F~�v��B�0�����7:
���{�K�x��9~1W��4���I{F��9�9����0
�(:�&�HsF����*iF���&�/y����q��v�i�(!���rK�t�M�z���f{I��CG�0���62r�+:F,�cV����={�2t8n��F�<�F�w�W�]e
�i;l���xS6�&?�l1SF��>(���KG�}���fn ,�7���=�G2+N���0�?��p�Q}:���tP�3=�{X��(����zH��w���qK��z(��:u�e����?�3�,Q�=yH���1���������1Y����0_��{���z/����R����$~S���k/�L7���EY�����o���`�+����ai*�C�m��`��~����������X�D���!M�t���2H������{�9S:���52�?(
�#=��]d������C�y������������4B)�!uj�1+3�Y����p�C[�������2W���Ox9H$��������=�RL:��W�
��2S��2�R��^�����Q���{�����k|F��i�X�G�bf����t���,�/��:�YExh��E�
��2G��;�������63.����]���C>�e(���d�	�����:y%_��?����W�iWY�f!)c,Y�^�~!��!����Ma��!���u5ab��H�h�Y&K{�>'dr���E�?q�>�_��r���-e�$Ku	;+��3��P��	�4����,cFFi����#?�!�;3�<GZ��S��A_�OX���7�"_�Sd����a
���������7�QNp��Iw�	������b�t!4�T�A�rd�
��7'�
|	F?�(��3����H��������$J~Sa���:E4
�AeNeOcOc��0~1�@C��f��i�VXa�����������N
*����s��;:l�|��WPr��r�K.���w���5
�"i���1�?�O��[�S��g�}4�� .t�D��Hc����A���(F4�aD�<�K6����Y�Ai�i�hixB�J=�5�C�Nz�o�E|��x�g�C�����F�0��O��AgI~1���(Za	y��
���@�03��E��4�t�y'�A��\1sI��^��BG����o���;��t��P���:
��w��P^H:>q��3��rA~���	���?|���L ct�(;!���#��
������i��iG�Q�C��X�v�����ne/~1�B���@~�Y���O�������t@�s�}�}�:����!M%��8SQt)��g�5�-��$rE��qtb��(k�#��y�*R�0����^�K����O�'iO]D�i@��(R�2{��=�%o(k�#�Qp��H':�(�w���2G���L���a?��WXA�;H�>d��<#oP\�#Kq�m\�2�C�2��<����s�?�����5~2PHz����K���q�w���gl(�!M�g�w��i�����n��nP�C����P"�t�)t�C{����:��~��RP/i@}�;������9�/���!�-e��?e%�2?fn)������2��<�X�(a�N�e����}���M"l�
���1�D��w�r��_����0��<�~~�/��/({�3* ��/�y��M:6�.����v�p�f�����i��NY�=����w���G����'����
���%�U������^R�(3��6�:��$� ����{�>1��2Z,���4�=�*���d��	{�	w�UF��_~9�:�)�1O�J!�:\����\��r�x�5���UX)��N�F����;��~��n�������*���;��:��S���v�s���kD�;�{��R�BL����
�'\�b�a',�������\����\��a��'��a��*��/��r
��?�\�!��.��v�m���t{;�:�%���;��cG����r�v\�	���:9���>;�*O���U�)�II����)�)�yr���{�f�#��rH�5�����ulS������kD}����!�<�����������'����'o�2���������
��k�|z����[omp���:��s��!]t�wG^��w�y��+<���8����)����F��K��z���mx�`�;�uD������K=����wc8�"��Zk���f���{�9�E9s�{�����{�����Cgr��^{������SNR�m�]��/��C.H�<���s����>��3o�����(�\-���#=�"���p��k�}|�	����2C���t<��#d��t�F���y;�)�yu�e��2�\�����"Sx��<t�A
�;��m�����o�9��N�����2���S\SN�l��Q��z9w��w3u�T�/��BN��~�����\W^ye���k�?���S����O��{)��#_Q��%�C�:��#�����S����/7�q����S��o��z�2�s�w_���7��u���!\�A�\���a�Cx�S�;^{�5��u\�l8E�!,�'�G����7���m��3�{��t��L8���-yw��z9����S�S�����g��)��������������O����8�Qp��6�l���<S��;�c���rH8�b����?(<�E�L��s@:E��-�
��;Wx�����{�N����>u
z��B��)�>~�}�������/RN�lh?)��u�#���&���U�=���}]��C���:��5b�����mLc��2Ay���"�z�GN{M[�u��;q
������g����o}8i;C�����J������
� ���?y��O�I���;�}%�N8���<v���e����;���'�<��0.��u!�����.��kE0:��3��1���3=,9a6�Q ��Yn�h
#��F���a&7�.�#G��I��1����R�3
��W����@�m��D��$\�)gD��������G��w3�����p��_��yF�Q�L�1e���5\>=X��hV-�"L�f�?�d����"��/i�d-z������3ZM�1{FDy7#����U��Q�NF��c�<,�`t�4s
N��:p�%$!��e�K y?a�#������O[��g�'�
?��n��2ZF�]��#�����n����;�c���C�C�I�	� @�)��lf
�2���,�i���t��/��3����=���+'q\���L����}d�r�����2�3;�;F�]'�A�I���'�H:��lf������Ag9��>��8R.�k�!`T�eA\��S>��kna&(����2KD��l�~f�XY@g�u(�1i��E /]Gw���O}�\�1e3�%\@���>2B���d&�2f���@�)�r��l.�7y/������������:$�%�Sw�af�:�6 �����{X~�	���f@������?;���y���,!k,�����13G|�'m�O���<u2`���l
my������e��YUfC13���9��s��/[�]<KX#��m�f�B� ��M���6e�����od(3��%���F��4�U�������4f�~����<����\���+���!L,�'~��q��q3����:��<�E/��O�f�C�/��LP�����?�2��@@Y6���k�cP��*�<��p�Ig]{|���.o�2����8��E�(@�X�?t��J��(�-�D��R���a�#�v�h�����8�	����Wmm�oP�d4���J��_<�XvA����@���������(��"�(�����gG���n��8�����4�()�iJ����:�;+��e54�,��sM��e��������Lc�"J����'��aS�y����&\,�J_���� ��s�v@8Y�D9Bq`i#��d�S<.���?��rH�g�2�{��A>�n!��'�2���������,�y��>L���vt�P���b���N��X����J5K��~����u!�nY���m�i���1�����0�D}0+(��9���z(�ACG�0��\����� ��C~��6t��<�
~�1yKg:� �AV����+��������$=^�ucP�qe�kfiAy�,����r?�%��������s�o�(�(<�=D�BN�[������x��-{ �s�}����YvI��4�>��`m#i�=���y�]�x/rC�����������,�3a�P��e���uD
n������pn�'���������,�.�������7�5���>�6��.���eS�En�2���#�b�E:��b�b}zP��Eg���@�����+���iH�\Zbt5��=��c�2J��O��fGhxi��@��
��"�Uz����2�����L�}��-�	
r����Tc�����t��;�O�N�)�����^f,��`��xd"��7
!
3�
�;:���`����X1��7f��*^��l�l��\��h��F|��"~\��*��M&�e�Y����
F��#�X\�=rf<���h8��S/���H�PN9	1�,0�����,�����0.0�@q���?��A����+�g��^�>����	q�Y�@~�?�=:t�)����LP^�5&�s
��?tp�yzZ������>EF��{��|�O�7�v�3��I��Cg��S�O�M�+:��1J��JHH�N+�`��)����a�9@�Q���3hC���>d��6v�����<{i��Bh��k
��� ��DqCV(�(%$���)V0+���a�+CsB\��'�EZ�����+e�v��s��{D��K]��p��+��I�c��X<��-�
�}vqk*�;+��G~S���_��~Lo_����9�ayK/i��4�E���*f�h<9��?h�Q���UKA�EC����tq�S�?�=uN��1��������t�������>���SfJ�m	~s�03�����f����
4a��\p�"�a`��C,���N�Fc�C��dS ��~�1���t/���(���o6��.���y�i�,k�7���q��W���A�\�%�����At(���t�;�,�&]��l�������L��*�_�X�Cy�u��
��p�b�b�B�(b�������G���qYH����;XvH��!,�c� ���Nd�4���A�tH:�!�P8��x�nN����-�	7�n��8��r��wR����p����������N:]����Hs��2���L180+�c���J��R�1{�3:�,�b��Y5�����2�a��:���c�)�VN2d) ���"�C}�	��(��kqY�~3�0��0C�o�"�(����+���(?	/�V�4'���&�@�>4�F�#=)��d3X�b"�q�t�~���t�Y�3��<�=���8=��'.a +�	~SF�_(s
e�0�N�e�A[�����k�
��\!��}8����`�I�V��?}������fy)�fG�x��C=M=K��d�J���4�X ��e9'����N*:�4b�������t{*j�&�a���A'�
��a����0+��k�Y�G��%�t������~3{hhx.��"�o��j:"�(�.(e,���������d-=��c��I#�,2��,�-�D�)��L��>������1���t����?!�!�q2�7���
4(`���-����:U�L
O&��a����v��_F2I3H�3=���44(�(�s2c��3�rN�7�����*�{��������f:F��2�N���dB�,q
"�I1��iE����ae���A�g���P�h�Yz���1j�?���.S�){�
C��Stp)��y(���8�#,}d�����
K���0��	�W�QD�CZ�w��� f��M]��1	�'��T�jv��`G~�|#����4!���1���P��\R^��{����(�K9���3C�����$�=�n@������$��2�K~��s2{E�i��}%�P��{v�C��<x#^��C�D��F��,{��(7�e7�y�#��%A�df�}��:|��7��)]�kNd��'=>�42���2@�Fy�lS���B��3������w"��Wg�}�\��!�| ���Y���M��L�R����$�?�K�ez6��qH;T�i���Q�P��E�PY�H�q�3i���f�3^�1�@������e`v�0�������/����H��|b`���l�\��.�}������/���Y��X�� Nc��YfC��)����s�]�O� �P�h�)�����'(��
��������xf�@pG�I{� 2FZQw#��{�)�JS���1�S�:�gFf����`�pf�{T@(t���d��%uTvXfE���M:T���TL���?*).FqC�GEF#F��g��{�+>BD'��n�F��*W�K��x��Si���PS��81��AA����
�E��#����#34��'��������y%N��
�NHc���	��N����"?P�h$	a�#���69��?����iF~O�f��/��t�x/�@q���k�m�=�l�?a��A�����g��0����;�s����3�O~�����.�������?���;��CG��"nh i�p�b ��3y o��P��t�A��^�E�{��|'mq�~�f:����a�6~�,nP(_a�#��^(]t��)�7+H[�H<x �;��p0�G��n:�(�����(�t>��a`&������C�J:�<�F��E}BP�P~gn�!���{�"����C�#��Q����F����	�Q��W��8�!�[�9@.�2�!yK�y �P�;�)s��<i@~�>��pR�'��A�)�c~�^�tv�{P��3d�A&��;�0:��'���#��EDxE�p!�<�����QB���t��yDg5�����;�p�uu��&!^���Q.y'iE����e8f��@����#��<���1���o�E��-�Cx��P��%�
��4D~)���td�m��A�$�(W@�a&<!����"-�=d��gO(mnI{��'���@e��
�(yE����GN�G�q��"��C�Ik����s��4�S7�F����N#��#�����A���i�=yN^�^�-�B�EH�!����y7i��	?��4"~�(o�?��I�H�������Qr�3��� ������7����ms�Iw��;;�����%MB[���
��u�3�Isd��L> ��}�7yo�(��(�������!����<�_�P�������w�s��|s�b��
�FA�2[S���Q�P�������#L#A|�� ����-a�B2��N��-iG9�����3�-�%
���nF<�^:U��wd�g(#���`�MZ��9�D�i��/4~q�;IS�����������a���e������r�����sw�Mc�
����Gf�V��s�0����M���:S\�gV��[H/��t��.
y�X����P��-����� ���N!�=�m�M �3�gs�;���+��1����N�����Y�����4���y&,��TMeN�����w!C��5S]���s(���iN�}��Y����X<aV���PgP��c&�K��I���2%���)$y?�c�L��� Z)sB_��a_�9�`�_!�E�)c��;afS�8l;a�	+~P���	��,�g6��YP�b�2&l��R���i�����AB!X���B�_�4Yz�v�L+a����9!DTZZ!�B,x���j����Rh6S��Y��*w!�b��6>\b�A�\+�o�� ��_�/!D2p('
!�����!e���\�D�a�z6��	!���i���!�"{��	��<7:B4/(s|�J��
a�Y�����L����My&n��7��MG��R��K.�����j�t�a�%����B$�Q>$,�<���7���^}�U�������Q��8N2��|��������m|W�o��'�����r�-���|���|x��_��=����g}��w�6�|s�����!�X�"�,�B4)s��Y)s|�3|pT�4�|$�s����hI(�tF�8� eN������������#���/�����������3J��W_m�{�W����(y|[��C���Z��<�">�~�QG���.k7�t�p��?�i�?��WQ����J����N��g��'�|b'�|�W���9!�G�\+�1e���.�Z1���|h����;Hk*X���c��V�f(td����
�.J��o���������]s�5�������l����������@-��>���`�_����]z��~�c��6��/���[n9����������oXl��l����'b�s�=v���7���r����!e��B�gR�%c���\�N�a�KUD�C���������Qk�oA�`��������2��������ng�}����O�:o��w�����
���o��N:�$k����GX���=���&M�d[m��=���~��w�}�g�P�P��=�\�������>�VZi�&��(��";t����\4�bs��Rre�i�t@9��b~#S�	�,�`�.�J?���p���/��+����?����!+=��_��B�RK>o����"�"{���YCe�\W����_�cn������{�q���B$2�%�b~'�?=z����t��cf
p���zw����{�%��n���/��s'N�^�z�%����q��J��6�"{
�tDf��x�������VYe_��&���r�������q����fw
�e�]��'4���(_8.�%��F����<��_���G��l�&�����j�%����'u���{��_������1!��2:�������������{v���������asJx�����4�����h} �,�m��]d#���?���+��_��m��u�'Pr�	��9u�%��������B�3���m��a��O/��[u�U����������~���P8�����7����X��"��F5���L�c!D����V
��i������G��h5��?���+���`��9���|rd�w���2d�m���7�
�E]d/�������FO�w����{6g�����m���~�6
��<��;4l��C��I^�c��?9,������Q����}Z��]x&��2��ry��P������'��t�\4v/�>������i�}S@���n���`���m�C=��v(q���J<1���2@�r���_���K%Q���=E�x��6q�5��m�$`I&m&��cf��
a�>B4)s��}�����k>e��sg��1Q�����o�@�2�1�{������s��g�����(y����r�)��j��{���]v�e~#xX���]w����s�G4�0r���t(z�

����~������q�E���t�e?���`m���~�d�LH�������G>�2���������?���!�A�'D�H��<;
�hqq2W���e,�c#v���KF!=|����]:t�8f�w������M6�d��"���k�����f����>��]w�u^y;��S�rP�=�y�r�r�-~_���h����
B�/0k}�UW�A���?&���#��t���=K��;����`
���h8
�g���\9��3��Yg��q�g	��q����//����,�B$�-��)sb���u�,Ya����0��6ac���D���Fo>>���Y������8,w��?.t:�)��(o_~�����O��#���{������1�c��=���\���Az8���a�y������q��k����}��7���zYd�3{n�yg���b������b���o���+�
��9��#d��=��C����M!�G����
:]j���r�
��c��g�y����x����9�Q��	��U��H#X��{8��ip����F�	�\�!���c�|����n���w����'b~����2��Y2d����/��,ke�k�e��&�/3u��a�7�00�\�d9b�3�>*���>���z3�@N6�_�N��_B�������.lw =�MF��h��i�k�s5+89���1�bY#'\�p�	^y�Byb��t.����>���9��f�mN�3�X����>"�~8:�tJ��23�"�RKf�����Y���`X��	_�,r�&3���	���(��2���\��1p�r�r(q�.+���=���~�����N����v�m���{��m�
6�3��>��w�
\$D���^|����l��[�5���>M�MF��R����|X����������zc����,��r������g������3g�g�������W^���dF��c������r�=����������;f�P���[:k���_�����[��X������k���w`�t
"Qe��=E��S�XF��D:(r�:�4��A>�������U�CRpD:��>:�R�x��~ �������s�'�Qc�+r�\_z��^vX��,����?��B���K��w���o���	�j����,E���5l������[9���-|����.B4	)s��}��9h�"���V���jc�L�g���?w�����{�U���T��[�\�����(^'�|��?8�d�]v��3����w�����~�43�,�F��Ht����������@N�
y�L����=f��'I��`�%�~Y!ZW�����������2'D������~4<��l�f�0f�cn�s����s�n��$`���-�hxv0��6��Q�P����l6K�9��Y3:Ja��s������#G��if���������A(}��cv���Y��?�s��e9&-�s$��i��	��>"r�#��}�E��u3��?�o��"!�����V
�>�3s�E�2��C���\v���Oy��;���9�3u���4
��|����(k,��,�D��P?�o����@��?����Mg{x	2�B����B�1t�43���{�i��r�f���)s��]�\���Q� ������P*"gP�K�k�������2'D�7��!`��+t����������{N���~/��H���!Dr�4K!�cJ|K��IH�kX�s����|��~F�;�e/���oo���b����`�������,������EI�7�ts?�_��t�)�x��{�
�f{���<"��h+��?>a"�H�
���5Zf�p��W\���p�6���A�r�����`I#Gmt�A��n��aN�92�e�3(�p�}�y��n����;����A�C���|��p���{���C����(�46�sov�,)c���(+k\�X��n;vl�w�qml"q������aY����+�e�i|�����oeq`�8Z}N�j��"�7�`v��x�B���w�#���>�i��O� �q>��������1�A��,�.���0��a�8�g�b��"���O�G�*�#�%����Y
4(�}�=z�?���VqB�[�2K>���$Fd�zI��3��w�E�l�ha�;-�l���a�����Y
�t��5a)#'�qLv\���7�t6�;J|�������F��7l:�����������^A�?t�����K{���>��s�D�yff��\�'������o:�t�Q�.��r�<��/��o�a�+s��k�_++�7���!�D L(�(t�vV�3�CXq��+����x����}�{�����G�B��w����Q�0~��|�u�9���|C�9����q?!�?����O?��G����f���,�����'I�e�]�����w�y��?y������"���#
*��+i/���}���
��u�Y�?���QdC�xG0�9f���y�&�����a@iJ.�}��|t�o�����7���~�-��"~��o�]t�E�>����e���V��1B��79�h�PV
m����u��[���J�\c���w�ayR��h2�4A3@g����^{�5�ajf��8���!^y���fF�]�j��|��S�v�y�?�Xs�7�|�1:������;L��rM����������C����[n�gD������|/���
8�!������r��[A:�(���������XJ��5y���k�l���j����`FIZs�5����������6����V��>&N\QR3�@<�C��$=�<'N(��ZaG<I_f����9������g�}����4|���X��2�������z������x(K�IX�+��1���LP|P��x9� �\(�!���JQ��K��E�|�w�v\(l��M7��+R�J��������@�3f�����S�P~�������g�s��%mqO\�w�Y��f(���r�-���O�>^9'my��!��3J%i���"�P��7����>M���}��3����|�]���n���_���4�Y�FXP�����3�?f������{���@�e2���?��}x��<���}���w�q{��=a	���`��`�����	d�]wY��_�����A�S���j�;�����C��u�W������
��:��A�J���u��]d����Z��`���?���k��
����}����{�]�+�pa)ta��p7�o�wa�����g�?����x7����Or����z���qs�<�s���?!p�1���|�C>g��Ln�'���I�'��d�m�[�x�������3����{���8�?�����`N6�gN�77��}<����$S<��X��c�{�q�tu������S&����R�	��;/�.N5�?�|vfo�����OG]�U����"��I���� �)�.���J'o��7;�i�Ud�taGVki�"7�#c�o�m&YE���;��d�����������Q��L�����$_0��a<o!�O���|��-���O�	_p��>�!�fV���["��s�'`�/�3��E!���@��-�}#��<K"��gP���=utH���p�{�{�J��L���1�_<�@ePWG�P_)4�~�W8���;.:�Tj(
o���_��r�b�R��0���$36��;T�
P�Q|����;}��13������%�|av�������a��_;���W���DY����z�+�{����wc����g_���8&��<��?�[��'�](]��������{�SP�a�e��J|x������`�2+���O��q�i�y�+<.�?�af�o�����y/;�0�$3�M��s�9�?
i�\�L��~����?��������~�GB�rJ
����"I�!L�x��1�m���`��a�m���'L����������q{��9�;�	����\��3W�M�O��spSIX]=��E�
sP���s�R��vuIP��]�����.���t1W���s���\mdOg3t(���?a!|���n�����!����q��K���{���x7g��7�6�^$�B6���f���6�o�����)���B�a�7���+�����n��
���'\�M���x|2�9=n��������sO�n�����w�9���6�x6f���1s7~{�!����nB|�������Q�� {3�a� �c����:�����^V]�����o�OW���~\nd���bf��pxs4@�����0����p�e��i��]�	�A�C<;�����i�����g0���M���W����!��n��g���<�9~���!�<��{�C<17��=�2�C<�/�;��=`�?1��
�h��������[n�r�����+u�-��������������R/��rj������	�w�:�)�iM
2$���?�{NHS�_~yj�����w�1u����{<�:��������r
Ij�=����T����/�0u�u�ys�~����S&�����G�����l���T��������O?������rJ�O��\��rJl�)���>����{�NK9����������CSN����tr�����^����t =�<���s�=��j�������)�D��:�,���������v
����������{NY�awJ��G>w�q>�w�e��SP������.����j��������o���n��)��D6�����?��8�o�rJ����S��M������������Rm�Qj�����wy������_~�Z}������6��9y�	��|��>=1s�W��zj�5 >=v�yg���|�Ij�v�iM��w�y
iI�����>-n����+����I��6��� �3iC����k�.��0���7�������9��)���V�++^ 3	j+�\g35���(Q���P�\[��c�+s�lM���vK���\�t43�L���L�����^�or�H1���
3i�-bf�Q�s��$>��=f��u�#-�P8������j�_FW��:�~��G�~��������?��O�Q����0���%�1�o�Y�����/7$=����3�:���4�����L�Ez�>����>���qIX�fK��I�O��Q:�7�b�����KYV�=�E<O�f�93o���46�i�r4;�{�E��K����F�������A����9�t�
�2��>2�����%�@X��V ��fd��f]�Kd',�?#D&���"��'DRp��jj!�C�\3@g9�:�;����G��!�\^z��~
�K%Y:H��e}��c)&{�8��P�P>8���������q�KY^��-������>������R��}��p�
�����XW��Kxe��9���F:r�b��%�%z��!N8}����'�p�	J�
�r�F&�����,s���V>(�,	DiA9��|��Pv��G	y/J�&�'Nx/P�x?Jv(���c���R~�9r���Y�RI�%�\�����l ����o������S�)����$�v�#i�������c��Q���_�%�}4� ��=�\�/���������m��������/�Y'�3O���k����8qHJ�V:��)bVa��q��A>5�?��R�����w�Af1p�	����Gu����I��f|�,J�	'��7e��SaU����p�}~lQrV��"�_nh��
��F!
QrQ(J����)f�8���r
i������]�}�����3(�3J��8@��a�n����B�f�e�wm���
�A:q�"�aBaf��_k���W�Pz����5AcV�|�dS!�Lp8�7� ���B��R��?a�<0�{f)�g��1^ac��YE����q(��|g���=�]��9����������GNK�\1P��>�\oNBL�w���0c�^=G�/�7�%}�'iJ�x�2LzS^��e6������}(y(�(��f`���-z�!����wR����>�������
e�����.���+�h�0c��>V�,|k-"�<���
-������rV����E6B��%�)"j���$D���B:����9�J���	����r��/���M?����V���8���w����)��5�C�N�((�������x2��K��q�G�A��]HK�����%K�"�f��7,������
3[������[��=a L(�AiK��a�D�?�O/<��	�8�?�E�����w���wa��8a�;x'a����!=9�x� Q<!�>�����|#��*��HS�7�H>�7q��'�H�����p�������l��<��AAo��&`��>M����1o�e
,eN�W�����{�"|�(�B�R�Z)d��(s��@�"�Jx&p�r��]P���a�)J;��qHK)sR�)�������_k-)s"7���2����w�N��#eN�,���	2'c)(���"7���6]�"�Z�%Db�I���I�	2'��h>��"������0=m�b��2'f�%�����.�K�
��������]��V�{:�1_��3�J!�3���`	N1�	)-K=)��CT(������V���5���i��b��gN�
�}��k`�N;Y�B{��h2R�Z)�)s����`����Z�b�c8I4�T�V���t���9R�fB�Y
�=R�Z)�R�Dn@aFq�2'r��������k�i�5���� en&����u��.)sBd���	�,c-��B$JQmmdB��b��Y#eN�������B��E�����B���ee^�MG��	QXXh��H
�,�,���B$FYU��C!D��2'DB���N��Z�%Dbh���#eN��`�%G�!���]��"1���E&!DS�2'DB0+�1�B�d�2K!��MUUdB4)sB$���	�<R��H��	!�C��	��'N��B��YNj����"��o�.2	!���9!B��,,�,����H�B��Y#eN�����m���~	!��]eedB���3fD&!DS�2'DBT�N��)S�_B�\�2����*DbL�2g!�F��	QPP`����/!D�a�e��s%Db���D��#�J5\��]�^&������"	��	��4������H���������>��{���������Vuu�=��C��4\�}���7f�{���������S�6(h#G��vO>���5�Ai��/���}�Q���/��H)sB$�,'M��B��a:�R������Y�h�u�]v�
7������K.�{���+\��a�:nP��;v�]v�e^q1b��p�	�\SSc�����=���o�s����[�������O�n�]w��y��R�D�H�"!8���E�,�,�2K!���������q�g��
���������n�������6�pC���������v�jg�u�����v���b�-f����_5��=������:������l�.��bC��~�����^�$�2'DB���X���_B�\�2�I��	�]�M�L��6d�?c��+���?��7f�����-��2^���v8//�f�z���:t���{��'�������z��_}o��Zk�������K/�d{����"I��	�|4�%B�d�W��y!cL��~�ss����*���������R�Cb,�d��+���9���+��%�\�_(r��
+�[lal��_r�rJ�---�������q��[!�D��	
!D�A�$�B$G�S��K }��7��SN�m���N<�D{��g��"��xeo��A
m/
��>�g�?�p��Xv��J+y�������K0��f�}����[n�'�D�H�"!�,AX�!�H}�X��h�e�,�D��c)$�s����{���m�v�K+����n��~����{o?��r��7�|c'N�K/��c�����_�X���O�[�n3)�B$��9!B�Y
�,,����,�H�	���%m����[o��N�v��G{ek��	���?�Y�����n��w������&�lb����W��/���;{%e�����w����w���~W\qE�r(DR��t�j��l��^f�<�@-�K6\3��ZB�����z�f�Id�zI���_�nC���9����5l������[9����w�a;�$��n��Ab~E3sB$�N�"Y��u�>���"�tm��,�
J�91?#eN��`fN'a	��M�k����"�����Y#eN!�B$��
��3�J������7\��%D�����(*�����B>�:P�����Iog_�����%�gn^b����"UGbn�2�J�2�<~��J�w�����5�(c�x�z���zO"7H����������jSR����,?_���;��R��X����O�&!���Biu�u	1�H�"!��p!�5��>.DbL�^��
�-R��H}4\�da|�>.Db��.2	!���9!�}��,�B$3s��;(Db��%Bd�z�B$DYY�u��-�%��5(s����!r���D&!DS�2'DB������G����e���i��I�~�"[��	��dY[[�B���:-u"1��%Bd��H��(--���;G����e����4=!���K-s"[��	�UUU6u����"	���D&!D�)�W��ME��	�2�:��'D��j����_"�"[$EB$DII�u��)�%�H�NZf)DbL�J�,��)sB$DEE�M�0!�%��5�2�m��B���n���I�T��	!�B!�|��9!���w��=�%��54��u��1����I�T�����j�����/���,'OVC&DRP;N.�izB$E�m"���H��(T�W3'��C��_q�]x>N�>���7���(tB�d�f�*,��!��9��"���H�ka*++���o�����������=�����Xuu�=��s��N;�;�`{���}���
����jGq����.���O>�����5�\c;�����E��i��=���^z���}������KO�;p�e��_B�\�G�;��Q�C�s���d��)s-��������/��n��F;�����k����G��aW^y��t�Iv�u��z��gW_}���A	<��3�K�.���p�
~Y
��/�ho���]x��v�e���~hw�y������?��UW]�.��}�z�|�:�B$�����$��5%I���)s-���"����u������c��Gj��G����W�Zh!6l����O^cV�Y:f��������/lO=��W�n��62d�����[l1�r�-��G�J�W_}��_��J+�g��������?���
�K1���
�~"7����
! /�R���G��H��vZ�p��M��L�D��R:f���������j=��|�����+��������_�	q���?�����[�gP��?�x��������7�`?������z?�L���ln��v��;���M6��?���?������^y������3�x��>��W:����M�66j�(�gaa�W��k���7O�4��7n\�{f
y{������&N��`.//��c�z3�NY�Q.Q��/3��������>����7�3q/--������xa�{�:r�H��M�xa�7�a������-�����U<���7��O�C|�'a%��=�x<)k�g�-���w�k���F������4��Y�{����{���n�C�=e��6���L�2�������O��=� ���<$�����=�i���� ���������e�<�I�H�.�sc���xf��x<�_�'W���9��)�C80����
�$����'|��LZ`��$-FS��������A�l�2�X�������������/�`?m��l�
+X��4W�[n9F���sg���V�d��]�WY��]����\���kx?�]�G����Uv6j�������p�\k-�����2=��������c�ulr�~������V��">�S���6.m1O����]�����x�t�s����2�G��^�_�g�K�x���������V��G^������r����U<o��un��$d)S��9�I�:���<�C����=n�}�N"<���>O��3��P�� �����0�i�N���uR�c�'����I��p�2�?�6O�K�[z���x<���I�%a�9��Od)�I!��C<qO<CZ�:i
�����=z�������\L�d�����V������\����Y]j�YEn�L�p~zYua@��-����r��Y���Rg �.l5��=�\�4���^��G9����������#WS��!?��4���.�~s��S����\����wal��d���>��.m&8yD�~���9hE[b��.�Jm��&�2��w��gK.9�
����������;��={Nw}��.���G������R��c����Wv�\������]]X��L���
����|[c����w���l�UG[�..��m��?�{���������S\�)r��r}�i�����kc�������Q����ys�>S]9kc��6�����d���e����<y�����k�
8�������-�t�y�P������rKy��j�N���x��Ry.�qY��I��\���<����iA���8�W���\%{�}�yE�e���~�o�N=�T/xt����K�����
�
,z7����J��������B��e�=y42g�}�ag���9������>�3���
T�T�]�v�l����w=�P�1�~���7�����MaN����\�w�����u8�[!Z���af��;��9Ge8�<����+�f�|z��YA��:Ds�����:���,����%-
�[o��GLN<�D�|����#F��u�^2��=
�
���B�H3k(\,�D��H�ci%J$�P����"����K��-�#hB���{��D#�g�������n��7����Ff�y�)d������MP��h60�gs�`"e��	JV (Tt�����7�8�?�,(X(m,3y����=9��6���Yw�u��8�<���~�J������p�Y>JS��3�%�H�;M�izB$��o�O�B4)s-HP�8����<���p�	^Q����-����o_��=��cp������k=z����\u�Uv��w��)Y���*�x=�PX
3}�r�2w������n��������_���;1�����-�H�J��	�E]$Bd����0���������"Q�8��c���>�q|��+9�r��W�36��O?��/�d��f�m�O����:=��n������+z�cy&�/��/���K.�Bx���9�[�m%��d����=s(uEE6�����Xu��t�f�g���:{�o�b�����u����s�����1����}�����dAYgf�=�B�)s
��27��W��Zk��$r�k{�������d[�z���)s�)h�e�@Y�����`v�Er��R��,�����@B4)sB$���	�,��h���Q�Y�h�-R��HN6e��"X��>���"����d��)s1�G�~	�RP���B$G�����%��
�-�"�jN�=z�
>�}�Q{������?��&�bP�8�R��M���+��-%�F&!DSi��3#������nht���?���O>�v�qG[s�5���*�G43EEE���B�d`N����
��#��"[Z�2������o��w�����v����G}d��}��g~����.������[��&?_��B$�Y
�)�4"kZuO�c���6�[�{��m�����m��vAA�������Y���:+zJ���e�|�]�,���e�B$Fi-�"[Z��'
����Y�I�&���1O�2��y�����b��c��	!����B+++�~	!r
�reUU�?�9�j��9�-Z������j&L�������=����������3�Zf)D����^��j������#��M�<�z��i?������{v�}����nk/���w#Ds��,�H�f)D��,�e�Bd����5N����~��������2q������N�"Y�i�Zf)DbT��e�Bd��9���v���/�}�������;���Yb��Go�9a�%
�"9���~!�vjMdB4)s���v�i~6n��v�M6����u���6�h��������W�,���C��H�6�F&!DS�2���K,a����m��6~�%{�8e���+!�NI�4U!D2����F3B$E����"e.�Y8>~�����h������������;#WB4(s|C�e���I�k��j���"e.�O�_��q������/n�v�a����GUU�M�:5�%��5,���e�B$F�%$Bd����3fX��m������[�z�����Z�:t�����2K!��e�%��"1��k���"e.�{����[7�Y�\�/!Z���B-�"aJ�gN������!�E�\G�����v�QG�>��cGq�y������B47:�R�d�2K!��M}kU�l�2A����.�M7��+s����
:��2dH�J��C�,�H�YkfN����,�"[��EpE]]��q���z��k����W_�������@�k��}�K�kP��WT��B��?M�LB��"e.�k��~��g�}�=s"'p������_B�\S�������B���+��9!�E�\�.]�����r��v�i
�c�=����C�,�B$3s�u�tB�$����	�-R�"X������	'��������_��={F��h>(s�4+ D�h���1���I�T�RZG�*!�/���
�<����2m�4�W�o
�3F�6���L3RVWTdc�z�<����V��.��af��[on���������XmJ�T��^�`G���Z��
���o����UUU�m�9��?���'��+��f)D��_*�i�B$F��������0u���;�`7�|�}���6f��������_�{��G��f�Ds��,�H�Zf)Db��e�BdK��3���K�Y��:�+r����m�=��Ko��6����]���#�B4':�R�da��N�"9���M!�E{�b���Xyy�UVVZ~~��m����i�]� ��g.Y�O���[�="!r���5�**��o�i=V]U�TDn���=s
�������j�\�������,c��8N��������y���R�9!Drt�6-2	!r����F&!DS�2'DBTTT��	�_B�$��A;!��rZ�,D�H�B!�B��)s`?Y��h)8��=�B���i�B$���G&!DS�2���(����������O7\�r)Ds�A;�&M�~	!r
�u���R��h;H�,��)st�/��B;����g������������aq!�NL���B�d�����)Db�UJ���)sf��(��~h��{����\`������n�����Yv��5�%��5(s]���!r����D&!DS�2��keee��cG?S�~1�"Ds�G����B�����G��H���u�LB��"-%��k��Yg;�����>�K+�~��;vl�J����:-�"QR�:���W �"[��E���7�|�~��g����eGqD�u�}�E��h>��p!��nd}4\��(�\
"[�R:�S[[k�'O�����Nw�v��Kd�%�\���?��}<Ena��4�����[d#D=��wo3�
[�������u_m5*��V��.��af��[on�_uw�����6��������%��#17hf.��q������/���������VRR��)rb�@�,�H-�"9�
%Bd���:�O>�����~~Y����k��w�z��6��l!�
@B$G���h�I1�-s"[��E�g��[o�s�9������O�}��~���G�\	�|0<q����"���mb���?�9���:MV�l�2���6m�������1���B���?������y�Q�F�����/N!�4i��g���#���>�����3�}���VUU�O�2����K���O�b��G�����?��B!�B$�����������c����;���]tQ2��5|�p?���k��k�������s*)Kg?����������^jg�y�?������y������{��;�����f���������={��~	!r

`�)�h�I1���X!D��i�$�+��bW\q��m��z�^9���>�:w��\0 ���i�����]�v��7�x�8��������?�����
7�`��w�v���C����?���r�-�~����o�?���+���������l������^��e���O��"g�4�8�r���[�!Ct|��
�):����Vjr�N���i��)hf.E`�
7��n����|��v���z�gAS�$P�n��F�s�=�bN�\m�����>�
9�\~������S�����JY:�`���y
"�1��c�������jjn!D"�����$�H��v��I�TZ�2�^9fa�T��g��Y��Z�����������ig�3B��}���~O�33���g/�������s����������~I`�{c����_�/�3
�tQj�YZ�@�Z���l*'���,�$��������2^VV���}�m��N�:y?p��IN�\a��O�>�>�~�EY����#���i�B$C5]��\���/���bp����;�E���&������{��p����������?��S������B��L�N�Y
�-�Z������k������'����{��
��K/�d�r����[^��7����������/�����o?�����G�������?�|o(}������w�a����(2[���~���t�I~�`xvA�w�����{�k��w��H'��k��c�9�/�4h��C���o���G��,//���{�!]�Xc
�_n�-��K3��>{'W]uU��7�a�'��=��i��z��3�m���6��=���8� ��p�-uv(rAi{��g���_�B���,\��M�U+s�������c��G(f��
f�=�T�e�3B'�|�u�Q��u~�(+�����W^i<���-��O����N=�T�o�����s�����X�q�~o����Bg�<���o��N8��,�r�����b��*�Z�n�|��(��������N~�5�cfw�n�k������{���J+�����9��q�T��T��f��^�z���,�n����������}���}��!�.�f�m|�PTT=]_/P�s�2������KFw�^�?�O��b~&���/"kZ���N>�y�J���_��b&�)�(�����[u,�d��+,�d&m���{�h�i�QBP�p�l�[l���t��qh��������@��}���>(����<���+�0�_�A�	q�o����J:a�#�,�%?w�}w?�}����Q�]v��+p[m������B���kze��G�����������}�w����:�j6't��rO!��	��9�w0���w&s ��������W ��������S3��3D��roV\�g��U�n��,=bY&�o,���'��T]�"�z�|��<�	��|J�r����g�����+n?'����s#��}]������������9�
���Y���p/�K7��B�-�|fdn���p4=3aI�E�<e�)�2�uc��#�������c��&�l����"��Fy�Q*PN���@��TEX2�l��_|�?�1�����;3���4�� (�,d�W��1{w�=���_JKK�q���0��`f��?���O+���p��,�
�,u���}��5cf3~�_��`f�"i����o�y�E���
�q�q#L�������m���/SF�[l1���������A���O������7���SMI��`O<	g�O�'q�?q�=K�B<	C�'Jw����0���=�M����3~��B��J�>��v�D<y'
#KL	#a�]���}���
��=�&��[�?�	��f)��I�x7����O�6���&>�e�p�W�ya%){�~�`����W�	7nB<1��BO�x��'��3�3�?�y���@xC<�f���3�I<�3a�9��	f��}�[�����?���������L<7a�J��q�K{�?���Nv
\�F������c�\�����o��ou.������:�l�z��_]T���U��W�U�rT����������������6�������Gl��M�����p������Y��ON^�8yk����;������.���u�b�[ni�..?9���W�2W�Xk-�R�f��g;��<r�5�E|��:�x9�������6.�&�4��#w��Od��A9%I��2�7��
�,�B]L~Q��?7�t���Q�����?=��-��R�9V`P�SS��l�����'�R��<n$�l�<�:�r�0�7��P�O�S��P��>�d/O�:�$>�L�e��b�1�I��|��Y!��s(��O��}��H7�C�	'���A&C�D��)��9�3��,���J|I�� �<d���p�:�0���>��x��:��������d�����/l�+3�\�L�I^i%_�����{Y�\s�2<��W��K��K/�e�E����[/�.�������5�
�_\y�p����Y�r�Z�{w�������3�+t�F���BW�������&���=nxOG�V�����O;'7�������;����`d�����A�����[��}m�)��\?e�k�G�|�w��X�����b[n����Z#]>X��S�l�������]�w�����_��k�*]z�8�������n���*p��m�%'��-��7��z�*w��o[m5�����d�����m�e��|/vn~r27���mm���wa��D''�?�0�q2P�f��	.�;x��W����������C�m��O��6��-�|��Xc�+��n��G�rXh[o=�����e.�d���:�2o?�e8�1<��Y����Sg�f�'�'�c\&�j\��r�L����J��3u��W7\W^yej�]wM]������~���S�RNp�o��\c����OSw�qG���N���0���;��p�>����SDRN@�='����>;u�x��������N�K
2$�������N9!�����N;��"��\x���k�����5f
�En�L��W�K�1jT*����o�sx���e0�����M���}�^mqq����?*�(q��z5\���~��3�h��:b���?�������n��\����Y�)r��^{��w
[j���k�'\Ni���SH�sb>��]j�������9��y�)W�a}}�=Sy5Y5� ����;{6�~;��ED��A���v0��h#��qa�(�>����;���������1d��w1���b�����~�k�(+����\2{��_ �R2��=��L����0��s\aP�Fv�k!�y�k��9!�g�������/�����33��\,�g�\�����{�1�Dx�z�����P���V`f)���,V
��^q��@�<'d�lS�2������f�l�0�d����M�gn�"���9N���N��������
�n�uS�1��f��1S�,�`�:�X�47�X��2�r������)s�:M��]w����`�
 �p�,�����=
)����F��/�XA�	!���N�N>��b��B}������Yr�0��9q��8���]u�U��'�����d��X8 �eR�R���L��%�I�T�R�	��t����ds�=��4�|����~m�n���O0���$K.��G�B�d/02�� ��p����K�C�C	�tKf�P�������wb�#�>�G�a�|����@\���x��
�-���h���i��Mu>F����GS���b+��������-4x��[�-�Y��~e�+���ia6>>3�[�m8�9��<v\����j����a����Wm���u��
���jS*��x#�����)s(@�>��?���e�Y����D���l>O@��q3n�I�bf��4���8��q�D�D��r�5�r��[Fmi�W^ye�43�H0���?�_���+s����kf[m��m�e���m[�r��6Z:+DN�2�@����_~�����zO"7�J�k���w�-G�+e.B��h
�3������t��fv�o��-P���,Q���f�;f���~���8Ie-���{����!�_W���_����D����#������p��"9*b��"�w��	�-R�"X�B�����R��w����L��I-���6�%��5��U��gj�����}�Rd!D��2��Sq�~���g��7����N~�3\�{l�r� �e���Z��2:��5S���U���e9�+�\�e�
��s�z��X�)t�{]M@��f��Y���vP�,x��������2B�,ES�2�3>���:��=h|P������-���g�$��%Oe��V��{����B�\5�E��b���?�)e�E������R�E��L���.��wwK���2'���YFpz%�
�s�\���O��<����"'�
jJKm���	���T��X�,c�����>M D�H���#���v�73K��m[�?GI���[;�/!��)/T��B��S����$�h*R�"XN��sg{��w�g
X�.!Z�T~���|	�(y�j�H��"���")�`\MM�������v�������8��������Gu�66]��"9\X��8�!��5%���R�l�2�r��W_��<�H�b�-l����Ze�U�^:!������M�2!���r���
��#5�)D�H���T��v��>�`q�e��]w����G*/O�x�H�<�'DbH���)sS�L�x�n����]o��f�J�����L�Y
�$yf����R��(���)Bd����CO8��/���o����hn��M�i�B$IJ���H�����Y!�E�\�,w�u�����r�]q�������������`!�#U���B$E��.2	!�����L�j�VZi%������I4;U������������>�")����LB��"e.����~��G�������W_��/�h�^z����O3(��)��������"���j&UG?���z��O�l�21i�$�?��+����W_m�?��x�����;F��h>
����@Gy	�$���-DR�L��LB��"e.�[�nv�gx%.\]t����^��������];�Z���B$�,�2K!��l���$�h*R������^x��������{���4K��-E�S�J�u,���2K-�"1����<BdK�W�P�Xb���g�i~��}���v����>����:!Z���j+�2K!�v��Y
�Z�,D��ze�o��}�������{�n��&���k��'��/��_�~�m�Z��C�Y
�0:�R�D)��e�BdK�W���o}���v�i��kW����'W�o�m��V���oF��h>
++�T�Y
�,���e�B$E�x-�"[Z�2�Xv���Zh���/P��^{m�����wN4;���[YII�K�s\�^5F�I!������$�h*�~f����JK������/�$��JU6������&��G��9����Rm�B�\�n���I�Tt�S�F�e��rJ����n�\
�����ZA~�A!�#eVWU�B������{����������n��w��]/�����*��&�x�B4'���Y�hVX��Y��V^�?�0!���O��p�v��G6zm���k!���l����/!D�)0+[\��	�mh����5^B$DAU�F��9��,��D?���f�N�"[��	�E��Y[-�"9Xf����H���$Bd��9!B�Y
�0�l��,�H�v���LB��"e.'[������Sm��i3}�@��&���
u���RW����H�����$�h*�IF��}�����~��z��gw�q�������n���N�J���d�d��V�B$���f��!DB$��o�E&!DS�2�l����o�l��m���^������W^��~�m���f��S'-�"I\�������G���)s��'��(r�;wn��\�>}l��I�,D�����_(D��j5P'DRH���)s:�m�����
=z��C�c���/��?�N�hn�����N�"9�8M�"�!��53F���Bd����n������k{���}���~�����������.r%D�Q���M�G��H��-�&�!��5mj���"e.���<�;�0�����������<��-��B�+!��|W�

�_B�$������H����?!�E�\���"�g��s��+���N<�D[|����B4/�'Z�v����������izB$��O��i��Y!e.b��	v����-/��b]p�v�y����^k�����������������<9�%��9f�W��B��Nkv2��Bd������B������JJJl�E�?����y������k��'
�B!��R�bp���>h�s�
:�/��x����(7�t�����q�"�BdG��	Zf)D����R�z")�~:52	!�����%�2���p-��2�����"�,b:t�����	!�C�Y
�0�l��,�H�f)D�H�������R������~������]}�����K�t������n�h���lW���~!#O�P!�Fb��];;���l���v�G�����@�g�}l�m�����=�X���S���Q2e�ul�Y!������������!�E�\K*`]t�=�����x�+seee��K<x���`������&N���B$�k��k@E��h�����-R�b���%�/���=���^�����/"B4y��1� �H�Z-u")R�?!�F�\���?l[n��_fy�9���G��^>��s�+!����c�G}�J����>Z,�H����[�Bd���>~�}���w���S���~v��,�_����GE�.6~�:�B$�k�j����a���I�T��9��+//���8�����g
_|q�e�]����������+,������"	�K�
��e:�@�lQ+�`�R�6m��0�������.�}��^{�JJJ�eo��)S���i�C/P�����MM����D����m��QV[[�����	�_��o���Y>p��_1oP2y�uj�o��uf3~�izB$��o��~!�E�\���n�������{�%��������/���;G��N]]��|�����{���~�`�~<��;�86l�}����^P�=�P;�����y���B�������C�c�9���C��^y��c�=��{��W�;��C�S��+t,���Z���%�!D�)�[��ME�\�^����-��B~��c�=f����=�����o��U�@c��������<�p���#�G��:�(��-������*����������e;v�;���2��n�����~�O������=�s�������]{��v�I'����on��?�E3t����x!�#�X��I�_"�"[$Ec���
�K*�7W\\�,�+Q�:� :t�_�	��Y�:��o��_�����a��}��g~��������~vn��&[k�����7���v[{������K��\sM�/3��&M�K2��`�����_f��"\�^g6�-�")�}�o����)Z)g:"s�����/y\v�e�A(Wp���1������{�?������m�M6�~����o��g�����������6q�D���x��^1C�������~��O8���m��6��VZ���=s�&n��6���{��>��z����)�/��wo����7�|�O�,**�Wii�_��b�2��;f�P*13�����'��cH;�1�,�������?S���N�,p�<�)��X�R�����m�K��V���MtJk�����5a��������Z��x��^�t;���&hU��{���Y�S��]X����Ms�naE��wn*;w�B��c�[����a�.��;�]�VVz�������s�`��/qa���9�O��]^��$�`OYtQ+pi��\ScS�����=���M���?���I�c]����;���b6�s�I�����3�$�)D�����L�|�L��;�!����d/hx7�9�
7���
�#�!?�����SVp�3��e�����x2;I�������i��7�J�1�x�.�(	��
��!�m��e�8'�.�3�3�?���;y����g<?C<��=����!�����_�'q��=y5z�h���~b&|��L<y6�;.���S��3�@���d/��m�"��d��r>��qo��:�kW����^V+�9�ad����M.�3�*{�]��[f�~YE���J'���X'o�{����#�._�,�����ee.��Os����d��	{��7a�f����{y������-�����V{9�|�ml�����[r��.�S�\a�/^��:TY��\��s�]cK-5��o�3�����T*��Yf�u�T��9�
���`�2��V\q���U�����O�.]f�2P�������t��e����\=]�d����)SJ��/�������;��
��&�z�E��������b�M����g�����X�n�����+��8Z�����m�UR���2�9�a��Q��r�����Y�I���R�C�d��:�g���'��LR�	v�=��y��*�
������2�'���p�xY�d5�7^V]x��}������v�F�2LY�l�2��h�+����� �k�Id���ihW�<yYua�p�m�[������u.�'"����D�Xg�l �3\;E;F��\_(���!m6��\M�J;����1+����^p�,��^��m\���$��#��i�RK��m[oW�'O.���_�I.�}���<Ok���R���y�%�e�]�j/�ee5���s�1����HV'��I�yC�����.��^�N���Zb�-7�Zh���B8p���
W��m���z?��?��7�*r��"�Luuq����>N�	��'�����|o�o����L����-�shc(�����9��PVy.��b��F[�O1�"e.����w���QnPt8�d�����X�UNs����O>i�������9�.��e������{eA���>�7�E�h�P
YR��:��
g9�.X����6b���� �<������!C\��w�G�o��7w���bY��"�<�:���F3��>��+�	���L���)r�\�=�X��`P�h��<� �t�q��/qf���������Wq�83�6���7�4��i��]�����=�y
JZ�3�97��p�,������w�7���)�������v�7�q�i	fOg��qn8��n������N!�����qs<O�'�C�c���&���*n�����3��?q{���1sx3f����+�>�t��>S����w�M�?�>�N.����=�c����a
���?�����3�7��+n��l�o0S/���|R�#��L�T�c�^�0��p$�A&��"?�����4�����'��M��a��������{�3n�a�q����sg�e9'���q��������Z{������}\��:���E���3f�NI~O�Zo������{:��GQ���	���"����3����	���������M+��b�O:z����������������{��O��e�
]�A9�/����-���/o����3�����nx7v���
a
�7�=����9�,n��f�ca���am����gW�����>�/��,��6#�^^b�i(��Wp���t���I�q�U/o���}[�{3��
��d���d��$n��
n\[�}hK	Wh?}����g���K�m?�����O�k(�������2�\�\�������d5�a�Ud�n
���\S��L7���/���{�n�e�^&q���e/�ffY%l��?����rKy��Q��Oy��������x���s�%���K���k����=>A��H:�
�K��e���x(�W\q����N��>��e�/���]y����G������??r�b����W�N?�t?����f�mf;:E�������SN9�+m��1Kw�5�x���3�8�V^y���]r�%^y;����[�����m�C��e���U�u�o�}�F6B�Fb{�������R�Wd[�>o/�X�R,�l��v��UW��EA;8l������[9O����������L��}�b%s���E��%��+w�m7�����gMe����'e2[�'P���3������f�\1���?>���S'��a�{��t�M���>�����+�������.���-���~���<�^9��]�F��,a�:��0!D���Oi�C�k��i�Bd���(L�{�W�8]%��#��/��"r1w�t�2��������9�#9��g��v�G��r(�,�D�C�@��/��~�a?{�b�r���o��/�d�7`&���%�\�����|�`��W�i��H�U��D��IP��02	!rMQ���$�h*R�"��:��S��^4f�XK�^��_�+RM����l��HN����O���f?�p�v��~����{����\p�W�x��D<6�~���v���6,��-qa��=,�D����y�t(��������_4;.DR�4Y!�G{�"~��7���#��n
��^z�%�������JTK��1��?�{�����i�\�pj�tW�����g����"�q�����C�g�E_{�r����5���Cm��������MA3s�xqd8{�e�e�_~�������H�J��������_�,|~�����o!D2��Q'R��(h�e�Bd�����-���v�I'���o���������{��H)E����W���B�d�3��e^B$����#���H��`Y��'��O����O��W_�G�C����.+��L�"�����c�B$E�B%�I�T��E�������3��S ���~��-�?���B�dp�\A'-�")���4K!�E�\����O�<���l����o��	�R�4K!�%��,�H�?j���"e.����p�	�����o�������?n8E�����P��0:E��(h+�"[��E�������o��[o���_��?�������������9!�������gN����9!�G�\6�q��yE�O�>��W/�f)���i��]ii�K�s��*FVF?����W-s"[��E�������=�X:t��~��6x�`�%�sB47)�m�e�B$JA��A!�B�,���b�G���o��:w�l�]w�?�r�����;��)S"WB45%%V��,�H�u�AWB$EQW-s"[��Ep��_l����m�e��Q�F��W_m{����v�m�+!�N�l��,�H�Y�V�B��?i���"e�7�1c����;v�QG�l`O=������v����B4'��|�#D��j?�I!�"{Z�2WYYi����?�r�����������+�f�m��;���:D��h>�����
�
��Y�K��	��}t�����9fE���{;��c�����,��������{{7:�R��3fXY���)����<�IQ5Z��
�-�Z�CQCiCy����3�<�.��R{��w���B�$y���k��I���Rg!���J�'D����$�Wz��v��w��O~��7����[n�#FXUU��5�A�,�H-�"Q��R��i���s���V\\l�
�+����{�9��n��1~���������2K!�e����R���2K!�Gk�b��o����|������5�\3r!D��e�B$��y	�uUu�I�T��l���2����
80����2K!&�����Y
��}��U!�E��l�i����2K!F�,�H-�"{��	�UUVRX�B����L���B$E�D���"eN������&M��B���Y�M�C�k�l��ME��	�_ScE�/!D�N��LB�\S;M�'D�H�"!�gN��I�U���9!��r���	�-R��H����m�S��	�Z��Eu��I�e�Bd��9!����J���_B��SgV=�:�!��5�$Bd��9!���B�,�H>M0^�I!��j��Y
�-R��H���i�/!D�a���y	�m��LB��"eN������i�B$L�N�"1j���9!�E��	Q\^n��t���QgV�kE�C�k���C���)sB$De�N6Q�,�H��YJ�,�H���i���"eN�����������"	R5��$��5u��?!�E��	QT^n�JK�_B��SgV��N�")*~�2K!�E��	�i���O�~	!r�kK��U!���Rm#���H�"!t�����,�H�f)D�H�"!J'O�Nm5*)Db��M�A��B$E�W:L�l�2'DBL����N��B���[�}�C�k:��12	!���9!�B!���2'DB�M�`]�kV@���3��ey�C�k�~��)Bd��9!�B
"Y\�f�>.DR�[V
"[��	������B$��Y,Db�fP�%R��H���S�}��q%Db��U�Z�B��:MV���2'DB��r��Y
��,[R�,�H��Kk���"eN�������<-�"IR�u�I�k��$Bd��9!�l�8��Q��"1\?��3�f)DRLy_�Y
�-R��H�]����hx��v�7�?B�����rZ�%DRtX�CdB4)sB$D���j��o���q���+��=�������e�����_�������go�����n�{�����o6��u�]v��������|���v�a�y�O>��L����+V3(DR�K���I�	Q:i�un�6��=(V������Z���f��M�f>���v�i^q�������������m�-��3�8�z����JJJl�������:i�������z�c�=��������x�����!uf3��izB$�����Y�l�2��0+��o��SO=e�<��M�0a���_��=�����~�����~���TVVFw����1�^{�5{��gm�S������*��b���}{���������m������KF6����C���b�ye����+r#F�����{�/�x�r���;������������
+,,��V[����cl��
>��5!�K\X��4�!��5e���<Bd������?�K�y���m��6�
���j����l�����{�v�}w?;:�(ct�������?�������{UUUv�I'�UW]e��w����������{����[�����'��
�{��G��y�T^��5������
`n���-��v��'������������Q��m���~~~�u���~�������H������?��?��"P���_�+���B$E^���Bd���������o��n��&��af����m�����p�
v����{���s�9~�{���w����{E��8fU���o�����}��^a�y����[��o�k���n��v���?�P�y�����S3.��e�r2r�H�w�}���7�|c�}���g�)������t��k��g���r(�����G!�+���k���Q���Y
�-R�Z�N�:Y���}�����/�5:�(W��u����8p��g���_~����j+������-��R����z�����[������M6�����q����}���s���
+XYY��=�?��RM�7�),��{���e1����M����������Emq��9�"�9������B2���������n��3Wc�.S�����
����x�s�o�C��`��x:��=lBy���4"}Bz�?�����2�q�l�,)_�-���p����/l�E�{�&N���R>(c<�s��S�y����������������_q�������.�.�i3���\��9�\q{��cw�q���`���Le2Sy�>�^Z�I&3G2��*�����-�=���w�����sz<���"��\�/HYAA���������=v��d���75nv��f�Ln�pST�����`�s���3�����U<��)#M/��3W&7\��d2�\�f�7f�]�f�����;o�>*��X��/����K+�\���5j2����=��2���n�����u���47\��)����J�'e��r�^V���2��|��������.��f����e����������J�g��PFb���qsc�mn��;1Sp�#2�a���An��?\�ej(_�o��nW�9�z���m�e����LND��'�|��v�����j[}��m��W��PY���6��ep�R���[�
$�������e�]�w�o������;����{������>}:���������`�J f�M�q���S��������fg7�����g��)�����oNQ�������W9�c����72��6q�%��Z.�i\`�S:*�[�����U�mk�M0���"[��5��wF��V�V�Tu�h�.���^�����k�Sp�-�������O�d�
�����r��7�z��2�>������,b�N	���c.����}��T�����p�:��]����y.?*]�'���m�������c��t1T��=��G��%��L~�;��=e��>��/���Q2�@Y��1���[o�A8�����M�QK]z���K�v�:����rKy�,n���~������,f2�O�'a
��
�������{���?���)W��Y.JZ03��g�"^>I��|R�)�q��U������x9�-�	'�$n��x�x<1�O���8���0���3f��y���C<�3��}w<����x�JIWV�����Q.r�+we���2V���/�.p��N��� g��|��p�)������*��]y.�����G>��*r�#��.\���U�n����5dx���(sqA��;�����|��8�Oy�^��s�K���u���W���r�UUN��na?����t��~�&M�������]��g��U���Or���Y����.j\���KLrv��<8p���J'�6`�D����&x?�L)qu�X��c���b�����
g.�A��[�n.���?�W�i.��&9��l�z������-6��u�+�n��4���y�E��rR����������N������>���6��U�>�k�1�VY�����J���a�\Q��2���>�SnC��,�eyeYE��'�$�C��Y�y7�7�*n��b�>.�����w�$���A&�����7�;�*����N"����.�����V����.���v�F��M�A������g�;�����o�U'����G�I/���2���tW���������C{�=m,�A�G�l����j����w~#c���k�{�v���c�[�����-r���"��o��?���#��g��I�`���t�������;��}��������N��\������y@����q29��G�������~�y{d�����]+���jmm�-�������c���Y����"�����%�6��0����=�e7����ib���^����0/�p�w��p���3�]S|}���Y�o?���?Q�(3��0Sn�5�_zY�w��m���qY�L���bG��/�_�r_�8�	r
�*.:�lr�e�5�~��gGq�,�t��,�M�c�%��P��	���7�4�d��qL=���;�[w�u�{��$Cfl<�@�~�[������$8M#�l��\��1��]��>�m:tb8h�N;��`�&��t�_uiO����]�B����t�M}Y(����Fm�Oa���w�u��B��������eF4?������"���S��
m��g���
]Yo��1>�0���j��H4��k�]c]on�<U��mW����
"��r��dtq���+��2��f��$�\s��C��=gZ�H���^�z�YF��P
�pR!{�x�g���CD9�����~�o�>�U���K�FN'M��l�6��A8(�\C���x�W�V���.��N;yE.�7e����w��+r�%?8���2�=��ABF��b�����,[��7
!rO�A-�o|~B��h*R�Zf>�!{���������VP�X������c��Q��"�=.>K�=�����4��>��s�(�����6h� �y������Gy�Y�G�����w`OAUM���"(c�
4f���v�{�����)hW?�&��=�$Bd�����Ew�y����*���.���L�����)9����^:�����wa��e�,�<��S��A&+����H3s�����j'�|�?�3�x���>��������{1��>��m4+ Db8�������(��;�7-2	!�����0��q*%��6��l
���;�H�)�(�@�h&Y�^'�Wb�=�p��Qt��K6��1+��/������kQ(j�3�>L��u80��(����k����v��=���������i�\����^[ew�����g.��Z{���#e��i,y��Sp�I��??��M�'e.Y�{.d�����4e���uC�������2����+�i�}���9})s9�vP�\o���m���R�"���� e��"e.y�K�l�����)}.�5CVY�O�-���j��l�/����2'e.7H���g�lo�L|@�\��9��m"!����n�p
z}��+J!r�kKz�F?������O�l�2'DB�������/!D����IQ�^�Y
�-R��H���������!��l����B�\3����$�h*R��H������:�%��9yfE���B�\S��82	!���9!����:t��~	!r�\)sB$Eqw)sBd��9!������Q�B���Y�o:YW�������5R��H���k�e�B$G�Ya'� DRu����"eN���-�4����"����4=!����O�l�2'DB�L�b#m�B�N��Y���H����4Y!�E��	QWXhee���IR�F��")��G!�G��	Q���u�^�B�N������H��^%�I�T��	�E����1j��H��Y�(
����4Y!�E��	��B$��P���S/T���	�,���C�B$�S��{jv\��(�]��ME��	�2��c��"1Rf�c��B�\S��4��)sB!Z/N�B$��O���2'DB�4K!��,{�4K!����V��-R��H�YN�i�B$�N�"Q*��i�Bd��9!"���*+$�B$I]e]dB����?!�E=I!��C��wF�K�s���-�~!rM�%�D&!DS�2'DBTU�����/!D�L��LB�\S3�:2	!���9!�+sS��_B���r���R��H����?!�E��	Q����XX�,�H�<��>�h�IQ�O����)sB$DaE�M��c��H>>A���H����h��"eN�����9!G{��H-s"{��	���YN�~	!r�Y.�e^B$�N�"{��	�:�R��������,��)sB$D��i6n�_"1��*GUF?�������$�h*R��H����m����/!D�q-`���F?����r�#���H�"!�kk��R"(D��*�"�"��U�F&!DSQOR��`����Zf)Db��*Gj��IQ���Y
�-R��H����N�"9�h�>Z,Db��e�Bd��9!��,���4K!��v�N�")t���#eN��(�:�F��Y!�N���H�#�:E�l�2'DB����45�%��9�l�L���"��_�CdB4)sB$D^]����E���P��B�\���	�5R��H��I����q��"	���}7-�!��5�_��Se�MG��	Q���-�d9�	!��e�:MO��h�l;����H�"!�R)��2K!%U�i!�B�'D�H�"!���m�:�R���4��u��I��,��)sB$De���H?��"1\X�D���"����d��)sB$D~m�UWJ�H�Te]dB������$�h*�I
�����_�;B$����?/�~!r���D&!DS�2'DB�����Z~r�K�s\�~���!D���z��$�h*R��H�T^����$/!�@'�
�y�?!�F��	Q:q��h�"!������B$��h�"+��-��R��K�{Tv�`���Xf!���e��� B$E��:MV�l�2��RQQa���.��������?�,�nc��������|"	8Goj�`")&��o\'Z
�
R�@���������?�V\qE+,,���;�F��I������6��g��	����T�}����,)"��Q��?���j��� en�����~�i;���l��7������t�b�~�i��-��P�����_��I��@�dP����:����)S��.���g���i�g�Xj9u�T;���-//�+q]t��=�6�tSoWZZjEEE
����������'���r�Yu�Y~>����fe�9���ZK�l�������������3��0�g��D�y����q�d��q7��b�^���_��Y�����6��+8�`���Q�s����E��>�95�W�������&�F��73�p7f��8��y����c�3�r�K7-����1���g�Ds�����\����!�2������5����9�����o��A�3�I���';����iS�ww.���q�2���}*���{�s/]&!��`��a���Q�����_n�2����������v��vX����I��������/c|��;�������C
���X&��{B���9�e6��d��3��VV������k���~�S��9���_e�����W��<�L�����������M�r��L�fr��,���/�����0��s�e���;z����2�2i�$�m����G�
��������O���/�[o��+p��(79���s����|>!��.���`���A�$9@2��dpf���o��V_}������2�2y�d�u�]gR�P�~��W;��3]EQ_S`�*v��|PJ��Nd��F�I��#��P;')9�_�������2�R^^nC���n��z�������_6p�@6l�V!�B�����X�())����2b�{������>��V[M��B!���[!K������O���t�m���3B!�B��)s(dk<k�Z���>T(�bAC�,P����s�o��NL2�������c��O,��	A��7������
3f���
!rA�A>��wp�-�h>43'�\Bg��}���Mf������������N�:���^k����\	!���#G�o"�m�6����|��w����n���
N�#�����+!����$�u��a���}���o3���{��W���.��2[i��4�,D3��9!���_�.�����������~k��O�v�D���v�)���{�i��s�?����~B���+���p�	����7����%���o���6�������}���|���{����������9!��3~�x���3S;��UTT�����B~s]y��v��G�
+�`w�y����.v�%�x7B��A������QE��
d�l�MCC��?�\�����6�pC���O��W\�]tQ[g�u�o��v���^B�9�����313�� ���n����C��8�?xr��{�d�g��^�zy2d�=��C�^1{����`I��v���$����[��f���;�����������o���^��o?�2�"�h��	1P�EdY3n_}��u���/���/|#�g�}�}���/�h��������[f�e��?��~�	eY�L�8��	!f
2x�q��|��*��3f�����:��z?������OZ�����s���N�U����^���5�\��1J!��y��7���������G���!W&L���i��f/���{������TYn���2��L��e��!�<+��)sB��7��7���9�?S�"�GG���?��W^�`]�v�C9��M����0��Q�Fy�P��|�����	1 ?�t�G����g��������&�l����t�o��=��#~�d�-��o����$<�w,��*��5�Sm����?����{��?}�^��n�������a@�K�.���;��K/m����PA&��T�{���
B�l�2'D

�h��a�
E��\�{'�t���������@���F
��_������~��Z�+xo��v�=:��+��@�oN@����{��0p�������=7?��
6�+wK.������~����p�"w<��@��a6��m���eB��3n�m�����o���~��G�^�P��+�-�o\�+���!����~��q/�h����p��-}I�4N���������a	��aI��hK-��?���G*fh���T�?����;��B�{�sh
�<�~;��`bA%�K#�3cd.� ���O>��+��7�t�]}�����^SL8	�[A�{��������]w]��D����]����� ��rw��G��sB,�|�������^{����r��3��y�scpB�


���0x���'`�����U�7ghC��e�B����LGdb����X��IF1��F�Y2������3��O������Zd�E����={zE���C�4��1��{4�����n�K�Py�?���������KKK�Y�������;�<�4��yF�Q��C���t4a�
K����q�+
r����Bt
_~�e��q�2�}��m�UV�{u��Y��^{�w����;�������#*��m ���` '+��2[d�����d0,��0!�0H���'����~!��0a��k��3��\���&����~���/l���?}[�?�^0!�G��X ��::l�/��-����Q�l�����6���^{m��c����=��]~��6v�X��1��L������/3��b�����4tt&y�('W�p�W��*4|B,H ,�BYJ������.[o�����b�-�:�����O�����{{7(i�	���~� �K,����"�t69�g�/����(���3��~�,FuEk�

bp����	�V�0��`�%�G��7�og�}�?���%���p��������!�@�� �@�c���L"��"w��|K�h��R��0�����.�8���aO���%��6�����>�
�f�����w��?(�����o�GY��2�h"{����O?��5���@������D)d�A
�XPA���>��S�9\���X,/F�a���H����;�g����Hg��!*t ��:d��������v�a��N"��UW]��e�D
�������n���}=h������18��k�������m�����v�?�9�>��R�D��y��������cP�E�����3�."7H��4F��a�$
��l��:�4 ,�	���1JI��R���rolh��E������%!,�D�c�������-�_�bA�2N�C~Xf�,5���� ��	|�Yby2��=��1k���w���4<������/�>��x�����v���0
!��G�/�bAA^�7�� ���(y���{���p�2J�8��V���%�,����=���7�J�,�����/nX��~	!r��Y�y�0��XC@���F#���46t���Z�Pq�1��hp�z�)��c��N!J�*��@������#��z�w��#
�f��PX���?�"��,���,�C
2-��7�=�Y���6�HP�8
�o�!�,�d��hX
�=�QQ��>6�<d�A`F�p���yd�G��������r�=�@�'d
�
���![,�d��I���nG����Xu��m J2(o������d;~��
3o��B��!eN$�3^44L(d4@��h4p�2+6a3J�� {�P��H7���g�6�f4dt0i|�)@�� F�9x��6:����4P<G�?����d+~	� ��=�O<��_J��h!+�<2�)�(|(T�������O?���#kW]u��`����%d�=o��fP��K:�(}�#K�Q�'��~#���)�"����y��g�B���yC(�`�w�}�*^��~�r���1N[�\�D�A��0�B�V�������(o�>�h3���~���R�B#Fc��=?��r�]v����������0"I�C��)�����Fy�5�(m��`�'
�Zk������kt<i�8����a�?�D���M
�X�A�8y��+�����g�
v��r�n��=�c��l�Y3�����G:���C�` �5�,#{T��:�&�(~2���qC�Rh!D�	C�d�E�����1Cv�����2�2j�(?����l1S�*���5��o�0��3�m��3��[>��w����&{�x�*|��La�]���6R�D�0�R�2:�L������CF*�������I��Q��n#BF����
��g��'�
J��(uax��5@'�����������~������!s|[�"#��3n�������AI>��3�f��)�t������#��AP�$��f��:�(;��C�^�
6���z��qh�~��\(��	3g�F~3�I{�s�C�88���w,]f��~���xP���b�G��H:y��3#�o�������7���Xq���$����$KG��������%#�4Rg�u�WC#�2��B�V(�����q� �!c�!
�G�A��03����O�=7��qd�$���yaf����0�������0��&�6�.8�,�GEP�}C�bK2�J��b�
d���������?�p�z%��G{�P1�=s�����?|g��U���(44t3>@�C�����>������N#~����o���d��}����%"tNi�8���`���CS�d�o>�,�	�i(T���.AVYr�	w�d3#��������IE���A,�b �Y8�e)&K&��o�����=8�����Y��#�#�r�"7�fco��(a��!w��f�*K�)g6
��O���0���u�1�+���������X��R�!`���J!W����7jUE�@c�~46^��}:�(q�I��������84,,�bC6{x��[n��/��0:�4�<��G�oU�82��&�&��co�G�����Dk�Qv���b��,��v����^~��FL�Af�9��oK�r����2���8����9��L"�,��~��m�]w��|.�����ybA�"D�_
�����r�m$r����f{������n)����;����� ��1���3�����-{���:�2(�Xp�s����M��D���?���g�
����2.5����G<���:�vB�14>���p�
�������`���������d+��P��MfZ�959�!"A�!���$q�"4C^RI@�n��&3�HYfR����Y���K7+�I#���?��.�
��}g������u�9�y��_&��1v<L�F2�8������#?s��)C
�M�����]�q��&cU���_j=)�N� ���X��%2;9��Jd�����~5Yl��2��
���	�����*O�&������S����_���Y�M�q�^��}5��:��o�7�5S�L��q����`4M�������u<9�T@��f�B,�����}�	�g�G�pe��!p�Y�+�lC��<7�Df��s�9se�M�e@�����0������)K��6%v��)��#����]������^���3W��s}fTD�j=��n!��Ir�a�9qT�o�g�v+���-r�@T�( ��(TvYBPB����������j�0"��+�P�]�
��m3�u��ee�H[���Q	��J�A���8�����D=N�w�������Zv�q2{�/�cl�Mt[��6��r�T�	y�%7e���H%����a3a�W�5r��R�����h~!����k�������'������iC�6��F���I��>���chV�bg���5P������T�k�����Oof��]&����� �?#("���b|���2l�1���	'���@D�`�-{�����v�ie����10�U���d��/�8���zu �OY��z���!��3�C�P"���Z�����x��Z��~5J�
��4HA�@�����{�y���S�O)�����^�oC8������A�'��n��d�����[����:2�w�y���b�[��M�ch��dW�$C4��
���-�&��i�z�?��f��y��Y&�W&
�0!��)�3�X������� �6v���u!��%=s}ea
��m#��D�22��I�0�>�����Q��(dJ�c����Z������7���E�-\��9rd1,�+7zY)X{pz�B?�0���Kc�O��
���������ug�yf�%Y5�q��r��2����*m��g�72+�oI��~S�)�P���0AE�!�������f�?y�y#K��O���uyA��v���@��'���_/*�s,]�q�{�L�6�8s}���D9t�g�O�1����/��SE�x�p���p#��Y;J�P����J����8tm���`�2K
�d/���R������/����!I�%(b)���W^YdB�^vl���e��Q���������V"�8��F�-���c<V\�|Z��ZS�P�s$}���8r��!s����&�Z�"���	'�H��Cv�.]Z�/�P8gMP�@=��)�9��s���I��\w��QEV/����/��;�(��[�������5fs���c,���
*�� 7��R1���S�o0��x2i���p�L�3���-��`d:�*2_e*��y���Cp�f��Y��Ub�����	�����>�D���#}J���rer��Y���a���n�����T6��
d\2D-��l/B?C����o�^mz
�Nf�<����w\�J�1�������?����m��EE�d��@���2s
����q�|�`���+9lB����������d��=������5���X�������5GuTs�Yg�I��o�NJ
��(@�M��h�c�����^�����0���1-��`��6���32�x��"{�	&�N��>��Y4�,CFN��ku��M�8�,�mDy
��!�J�B��_�,0���M��Vrt��7�x����'��b��������&V:V��2*\!R�b����y]����*�Bx�����C�(�����������������[]6@�����};���5k�"�O)��&�Rd�2s��N���!f8h�E�Ci��T��J�j&���1�	0"m2rd�|��*��}F�u�]��TL���g�n�6��zq��rH��3�C��^7��92(���X�����{���X:���"z�d�d���2��)��F�s�\���[bY��Bx$H��6G�x�)����1���G#�GR4�EN�(�L�}"�zr8tH������0Pf������aJ)*����`��t�5���T�9����4A�O/���9i�5��\��m��{��a�Zo*�T�3��Z�X�/v8o�ot��AdM�l�`�����/�p���L���i!���//�I�p�Q��?�/CG'���*��^��`aK���6����*����R+F#�
-aR2��"Ecm���q�D-
^P~b8
�L���#W
E�e�^�}��-�����l��0�lJ�/����\�#�G���*/�H2)xr�A�^U����3��\��:�w���,��^��`��q�,�A
��.��'��}������K_����<y`���K�Rr��;��rMr-�n`='�����2d��@�kU]B[����z��U�*h�:(F����A4��p��-�k��S�KUp��3�8�������~w��U����S�"��U2�DY��F����H��� ���uM�Y	�X��z �dJO�����{2���/�;dN_����W�L���_�T9�2@_BB�$�|�%�y�a��`&�r|���dX�m"��'�)X�B�Dfe�����8����!���8s[+e�j����]z���TDYCr��)%�(��9Sb�M�W������z1)�jp��=��n!V����~S�3���!%�H�o�;�7&R��!��f�z��H�e����S��,�R-CO&O�\A�W��F�Bx0"sV�F�K������^{�2G���A�&�GqDq�z�[�I�����a@��B�+U�m!l��gnA�p��a,�U����>�8c��G�z��~���,D����<��]G�ht
P���F4+����������G������1;��^r��'�����S���r��J���` ����G����j���	�W�x� ���A
d�f��[�	2�y����+�!��O�Z��"��3[��y�$��>9����3�������Co�9��7%�U���`!l�$3�(R8��'�\BeWF#����g�Q4m�����Sr��y�J�N��&nP���	0�\�r��1���Qb�(�0�a���y#?�4#�E�
2�C�P$/ �2j�i�B2:
F��6URV�2 ��
+�8��^M�$<�����0������
-1P�#�H��z[z���z�� ��s���d�',(���+W�uP�������k��d��iS����d����K�0����6�1�F����c������W��DR������g���Z���kF��5��v��X�v�m��j��[���b��P`q�B?Q�����
d��c8]2b��:yU����D�O:��2tD9���^��>6��@	��}ct�]�%�2�2q�:��d�}����G]�y>�F�B�QeP�B��|��G�1d�:S��e���Z��kN]fIe���J!������[J&��zN9{��}@�N�8��~�3���7�d�9o�:'�\�_Ifn#��l���j4Q|���m%��R.���A)�F�Pjz(-Q}��N�����3L��I��q��CI�"��x����9W������-�~���1�?���?X��6�_}��3!�o���@Q}�c���g�$��#U�@�g�'p��K6]Y���P�
�����e�����A
�o+k����@���~52����	.�kt��$'P������<!_t�@�j�6�-HB���T
�s�1e?��%pBl$3�0�(.�I��H�h��	Jo
�C�T��~�(}��L��y�w��w�%�d���^8Y7%����q�(J����)z��2z�x��cAqU%���0����ki�  �����&G�D%Z�54H�19��iudk���}��er�8t>����cTr��(��}��GY��9U�"���)m�j�e8��	�uZ�=��St�J�F�w���E~|S(�*9UE��#W�+dL�E8])�&.xi��l�@*�Q	��J2���8r!��H2s�N�R*����?��h��R�e
(=o�6�F��:��s��1��D1�9��bd�D22e8��w�nre
d����T�V����V��e7��,3��}��e�am7m%X�E��9��iY��O��G�������%�L�[�dI�l2���Y�&�/��\d0�#���'����je�5�h"��m����'��@Qj�A��������KQ���$�J(n]*H�����z��f���B�����E9o�Ef��-���g�}�������(��'�E1I~��1�\c6�)�	F���������1�����(-
,Q��op����?%?�W�AE�+��C�Y4��0:��2+�������5�3�W�,��������V
�ct�eAgj�s|u�*w�-�~��	h����@$��"��Q���%](�B�d�V�^]�8��3�+m�;e�e�]�L:�*&����e�8u2�t'G��]�{����3(�9�h�����W/Gu�(
���x�E��m��B�*r����X�x����S2�|�z������h��h�J���w�]��.`<��3���W��e�|1H��YP��'W7�q�!p�8~J5aF��t��a]W�D�sWe:�~�l��9Yhe�&&��!#?�V�Th8��Ge��R2qUf�������8s*\���'�g�g�Cu��������B�0}���*�
�k���������K��H"#�W��Q�!��}zg8j^��()��8m��c(&Y;hU��G%]^Gq������c��^���#�=�'z��Qr���J����]U&���N�%�OP�dXC�U @��Q���(�eq8kUV���2c���&K�zd �@z�q �u�������;g���E�������tz�=���Q�2!�6������4,�D���k�8�
Z7�3�g��iE�PN��w^1��x�^�'��n�v(�0TA����RKQE�!�N@���J��_�7*2��b���rd'N�����cj��@�j�����//�p�7�$9�����	��x��G���@�`@r��v�
���>���K`��j�!}o��9VdW)���^�
='�"�M����L2V[
KLtv�������A_#�!�����������dj��"����t|��L���7G�U���B�0E��p�(��M09ozvf��UJ�(FF����B��������z�]w�}6�C�,h/;P)���)A6O�D?qC(�����'S'x�s�����
�X�=%{��*{��08Q�l`�u�@�#U�X��~�ly�\W�@v����f��.=�A�{ji����u!����kgN�_y�E)/k�p��Hr��i�#U��x��fl���'�&z)C �`m%���I���
��2*�����%�B�p8�:I�9���W��D�e��A
�y�gp*��P�j�5_����9�!&�,1�_ D@�|�}��+���l��&{!_2r�$�_M��2Pq����`��B����rDe��"��p<e�JN��P@�R�B��3$�i���_)X5�l��&Fd��t�*C�B��hr���X�W�1�Qi����~�_�����n2g�a�b�D2��������$��
E�`���(�"�<b"XRe�
9�+n
�Z�l=��3B�����9
�����}��|���%g���#��zpz�(;��)3�����q�r|_�#�:��t;����8���d�8��zT�F��:����)�pC�7��u���^a`�:�������Ek��[�&�
~�������P��y#����������q��_����^���I6�R4QI	E���<�L��r�����Bx���_�|y�XG6�;%���RO�,�fK0/^\�NV��_|q�����
�4k2���>*M.������n*����;��B+s6�RPS�S ��Z��B[��t���;��J�D;*)2�Gvn�������^�4X(�q���:�$����/D�������%K'[�Q�4iR�����Y3p�"��(U�3qV�Q)�"�����6��^��#gz� ;n)�LSeU���r"�!�����=s���zzj�	R��SV�{JE_�#��QeJYV-���1���v[Y�M�cq���������q�������6��w_)��)�|�,0��R��r�:r������p�4|+�����3"��g_�W��rH2��RF������S���[�`AY*�t��p�"{!<�(s���!;��c�0��Q���~D'.���!_�M���W�o
����
"�
!l9�i��B�/�3Bxd�oD)���1���!�B��#�\a�P��$�B!���g.�B!��A�DB!�B� q�B!�Ba$�\!�B!l���!�B!�m�@	!�-��w��|���m������_��7{��w���<g�c��I�&5^xa��g?{�����{�����Nh^���6�{lY,�C���#6������a���x�9�4������;��~��v�m��E���[x��>�tT�{������5a1��������������7��;wn���=���9x���������+���;}�[��<��O_w�����ks�G4�|����g�m!<�$3B[�?���!�����n�O~������OZw��p8<��6^��;���a���{����QGU�z�7������\<�iOk��c����}m��������W^�v�e���I�����O|�y�?��b�����Y��O}�S�o��/y��_��<�G����!���'6&Lh^��m��hc�Ba��������B�2���o��6��O/���;��\t�E��!C�;���d�v�i�r������/��y��_]���}�k������/~���%/yI���=��_��W������7?�������������_�jqFd�d�����k\v�e�w���b������5�\���/iV�ZU��������y�}�5�^zi9��O}j��g<�\��[ni.���f�w,?���f�*��q������f��)�g����^�P��/o~���5/{���}|�K_*+�l�	?��%K�=^r�%�M7�T���'~��������o��y�^P������X�fM���>w}&P6��x���y�m�]��}�s�kn�������n���?������=�<����\<��?����z^,(��a���+�'?���y�:D���o��O?�9�������c�=�,�Q��{���-[V2e~'�=G���|?��w_>_f�3����+�9Oz���5�.=���i��+��������p?������\\�3���l�gv�L�7�������_�g�3C!��$3B[�]���x�p\}���@�1��"�kN��y�=��������e�qh��3��<��r-�u�]����?/����/��{��y�	�2G��+������������~��P�����_�r�&g����n>�����!/�d_/��7p&O=�����s�8��������59������}�c��#���'�x��G��������8����p�9$�����|�����c�9���?�Q�$���a��q�8F����Y����N�������(�g�v�i�9:O),������'��o������g�o������8�~�����h�����O�8�8~>�3�={vy�����m������9��?~���+����)?���Ts� �������\��>S����,����b!�=��!��]�������p����e�:]G����{v�Ag������?��u(��uX:���o���uV�^��c�=:]G������C�v.���N���v�a�[o����4iRg��9��'O�t��r����|�f��N���t���g���u�]���o��:]�g��|�3����;�|?s��r�k��-�v����#���q��y��;]c���:����u���1���q���n������+������7�|s�����t��������������s������L#F�(��uL:x���������3����k:K�.-��j��N�9�t�N�iz���^v�e�����}_��[��ul�}�G�?�����u���Q�Fu�M�V�es��T�����3�6lXg��v�����,���cY�u��N�I*?g�[�i*]��\���;Y�xq��������9�������d�����]��s�����������o��3:]��<C�������1c:]��|�.����:��3�uR����g�b�����sB!<<��!��e��]c�d�j�Y��t����23�A����z�k^�>C2��ES����y2(6��z��rJ�r����(9�c�Vq=����?�t-<�xu��Q-!�i����E�5���W�i]��d��L�W�U�L9�{�Q�,��C)�T
�ut�O<�d�j�L&O�OfM��,�2���Q�#KU?�>;������z��{��P>�f�P�!m~���6�}�+^Q~'~V��������{�{�~������<d�>�����g�x&�o�����B6�}�{_�v����m���l?��V�:_�9F������>����{����3���#�\!la����3g�2�:L�m|�q^��v��u�t jo���X�:R��hW�x�Yg����c�����P�����'�^�\����s]��fZ���������~��z�����~�!��)o=zt*�Z�]y��o�c�����9�~�)p��~81p�����R&�{��^9�c��)e�J\9�~���U��q�9�'�|rq��V����e�k���S����s��)%��f�?��m j	lu
����%�~g���qr�7���Bx(q�Ba+��m��p�����@����1�e�����\{���XY�
���z���o|�e���dy�a1�9s�^2)��^��q|�C����/&r���+$����y6�{WCG��q>6Y8�{��~�|���i�)�L�?��ug4�1�����@}v����s�=e��
g�P�sn�����y�;��I��3�G��u�����.��C9��]��w�I�3�#3f��$�����h��:�s�|
�q��8]�|�3�7��M��Z}���g�}��f�;����-���#�\!la2��7|���v��3f�(Y�ZNgS��x�_��`
��u���#F4C�->�*�!���o�����k��������;�����g���/�bL=�E��R�S�}�K����U�d)���g����99��3J9�a$�!����<�y���^2r�'O.�YV��!&�#�c3-�3��Q�T���&����k��q���g:j���l�c6t�m8��p�?��C�s��umB##G�,��|��Q�i��#~^�W����Y/�����	5X����U�z����{}�^e6=�{��`�M����5����Bxxd��B���l�5�Q���u��@�}{?��������}68��~u��]jX����}n�y����qu�@���������J����}�@�����gk�����s������<��u��V����m�z��w���C}o�����
�x�W�o�w/����~f�����B�t���B!��6H�,C!�Ba$�\!�B!l���!�B!�m�8s!�B!����4���I.�p5IEND�B`�
#27Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#26)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Fri, Feb 21, 2025 at 12:57 PM Ajin Cherian <itsajin@gmail.com> wrote:

On Thu, Feb 20, 2025 at 3:08 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:

Dear Ajin,

I compared the patch 1 which does not employ a hash cache and has the
overhead of starting a transaction every time the filter is checked.

I created a test setup of 10 million inserts in 3 different scenarios:
1. All inserts on unpublished tables
2. Half of the inserts on unpublished table and half on pupblished table
3. All inserts on published tables.

The percentage improvement in the new optimized patch compared to the
old patch is:

No transactions in publication: 85.39% improvement
Half transactions in publication: 72.70% improvement
All transactions in publication: 48.47% improvement

Attaching a graph to show the difference.

I could not find any comparisons with HEAD. Can you clarify the throughput/latency/memory
usage with HEAD?

Here's the difference in latency with head. Again 10 million inserts
in 3 scenarios: All transactions on unpublished tables, half of the
transactions on unpublished tables and all transactions on published
tables
Conclusion:
The patched code with 100 transaction throttling significantly
improves performance, reducing execution time by ~69% when no
published transactions are involved, ~43% with partial published
transactions, and ~15% in all published transactions.
Attaching a graph showing the performance differences.

In these tests, I also see an increased performance with the patch
even when all transactions are published. I will investigate why this
happens and update.

regards,
Ajin Cherian
Fujitsu Australia

#28Amit Kapila
amit.kapila16@gmail.com
In reply to: Ajin Cherian (#27)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Fri, Feb 21, 2025 at 7:57 AM Ajin Cherian <itsajin@gmail.com> wrote:

On Fri, Feb 21, 2025 at 12:57 PM Ajin Cherian <itsajin@gmail.com> wrote:

Conclusion:
The patched code with 100 transaction throttling significantly
improves performance, reducing execution time by ~69% when no
published transactions are involved, ~43% with partial published
transactions, and ~15% in all published transactions.
Attaching a graph showing the performance differences.

In these tests, I also see an increased performance with the patch
even when all transactions are published. I will investigate why this
happens and update.

Yes, it is important to investigate this because in the best case, it
should match with HEAD. One thing you can verify is whether the
changes processed on the server are exactly for the published table,
it shouldn't happen that it is processing both published and
unpublished changes. If the server is processing for both tables then
it is expected that the patch performs better. I think you can verify
before starting each test and after finishing each test whether the
slot is pointing at the appropriate location for the next test or
create a new slot for each with the required location.

--
With Regards,
Amit Kapila.

#29Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#20)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Some review comments for patch v14-0001.

======
src/backend/replication/logical/reorderbuffer.c

1.
+ *   We also try and filter changes that are not relevant for logical decoding
+ *   as well as give the option for plugins to filter changes in advance.
+ *   Determining whether to filter a change requires information about the
+ *   relation from the catalog, requring a transaction to be started.
+ *   When most changes in a transaction are unfilterable, the overhead of
+ *    starting a transaction for each record is significant. To reduce this
+ *    overhead a hash cache of relation file locators is created. Even then a
+ *    hash search for every record before recording has considerable overhead
+ *    especially for use cases where most tables in an instance are
not filtered.
+ *    To further reduce this overhead a simple approach is used to suspend
+ *    filtering for a certain number of changes CHANGES_THRESHOLD_FOR_FILTER
+ *    when an unfilterable change is encountered. In other words, continue
+ *    filtering changes if the last record was filtered out. If an unfilterable
+ *    change is found, skip filtering the next CHANGES_THRESHOLD_FOR_FILTER
+ *    changes.
+ *

1a.
/try and filter/try to filter/

~

1b.
There is some leading whitespace problem happening (spaces instead of tabs?)

~

1c.
Minor rewording

SUGGESTION (e.g. anyway this should be identical to the commit message text)

Determining whether to filter a change requires information about the
relation and the publication from the catalog, which means a
transaction must be started. But, the overhead of starting a
transaction for each record is significant. To reduce this overhead a
hash cache of relation file locators is used to remember which
relations are filterable.

Even so, doing a hash search for every record has considerable
overhead, especially for scenarios where most tables in an instance
are published. To further reduce overheads a simple approach is used:
When an unfilterable change is encountered we suspend filtering for a
certain number (CHANGES_THRESHOLD_FOR_FILTER) of subsequent changes.
In other words, continue filtering until an unfilterable change is
encountered; then skip filtering the next CHANGES_THRESHOLD_FOR_FILTER
changes, before attempting filtering again.

~~~

2.
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100

Maybe you can explain where this magic number comes from.

SUGGESTION
The CHANGES_THRESHOLD_FOR_FILTER value of 100 was chosen as the best
trade-off value after performance tests were carried out using
candidate values 10, 50, 100, and 200.

~~~

ReorderBufferQueueChange:

3.
+ /*
+ * If filtering was suspended and we've crossed the change threshold,
+ * attempt to filter again
+ */
+ if (!rb->can_filter_change && (++rb->unfiltered_changes_count
+ >= CHANGES_THRESHOLD_FOR_FILTER))
+ {
+ rb->can_filter_change = true;
+ rb->unfiltered_changes_count = 0;
+ }
+

/If filtering was suspended/If filtering is currently suspended/

~~~

ReorderBufferGetRelation:

4.
+static Relation
+ReorderBufferGetRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+ bool has_tuple)

My suggested [2-#4] name change 'ReorderBufferGetRelationForDecoding'
is not done yet. I saw Kuroda-san also said this name was confusing
[1-#02], and suggested something similar
'GetPossibleDecodableRelation'.

~~~

RelFileLocatorCacheInvalidateCallback:

5.
+ /*
+ * If relid is InvalidOid, signaling a complete reset, we must remove
+ * all entries, otherwise just remove the specific relation's entry.
+ * Always remove negative cache entries.
+ */
+ if (relid == InvalidOid || /* complete reset */
+ entry->relid == InvalidOid || /* invalid cache entry */
+ entry->relid == relid) /* individual flushed relation */
+ {
+ if (hash_search(RelFileLocatorFilterCache,
+ &entry->key,
+ HASH_REMOVE,
+ NULL) == NULL)
+ elog(ERROR, "hash table corrupted");
+ }

5a.
IMO the relid *parameter* should be mentioned explicitly to
disambiguate relid from the entry->relid.

/If relid is InvalidOid, signaling a complete reset,/If a complete
reset is requested (when 'relid' parameter is InvalidOid),/

~

5b.
/Always remove negative cache entries./Remove any invalid cache
entries (these are indicated by invalid entry->relid)/

~~~

ReorderBufferFilterByRelFileLocator:

6.
I previously [2-#7] had suggested this function code could be
refactored to share some common return logic. It is not done, but OTOH
there is no reply, so I don't know if it was overlooked or simply
rejected.

======
src/include/replication/reorderbuffer.h

7.
+ /* should we try to filter the change? */
+ bool can_filter_change;
+
+ /* number of changes after a failed attempt at filtering */
+ int8 unfiltered_changes_count;
+

The potential renaming of that 'can_filter_change' field to something
better is still an open item IMO [2-#8] pending consensus on what a
better name for this might be.

======
[1]: /messages/by-id/OSCPR01MB14966021B3390856464C5E27FF5C42@OSCPR01MB14966.jpnprd01.prod.outlook.com
[2]: /messages/by-id/CAHut+PtrLu=Yrxo_YQ-LC+LSOEUYmuFo2brjCQ18JM9-Vi2DwQ@mail.gmail.com

Kind Regards,
Peter Smith.
Fujitsu Australia

#30Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#20)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Some review comments for v14-0002.

======
src/backend/replication/logical/decode.c

1.
There is lots of nearly duplicate code checking to see if a change is filterable

DecodeInsert:
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
  ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
- buf->origptr, &target_locator, true))
+ buf->origptr, &target_locator,
+ REORDER_BUFFER_CHANGE_INSERT,
+ true))
DecodeUpdate:
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+ buf->origptr, &target_locator,
+ REORDER_BUFFER_CHANGE_UPDATE,
+ true))
+ return;
+
DecodeDelete:
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+ buf->origptr, &target_locator,
+ REORDER_BUFFER_CHANGE_DELETE,
+ true))
+ return;
+
DecodeMultiInsert:
  /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+ buf->origptr, &rlocator,
+ REORDER_BUFFER_CHANGE_INSERT,
+ true))
+ return;
+
DecodeSpecConfirm:
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+ buf->origptr, &target_locator,
+ REORDER_BUFFER_CHANGE_INSERT,
+ true))
+ return;
+

Can't all those code fragments (DecodeInsert, DecodeUpdate,
DecodeDelete, DecodeMultiInsert, DecodeSpecConfirm) delegate to a
new/common 'SkipThisChange(...)' function?

======
src/backend/replication/logical/reorderbuffer.c

2.
 /*
  * After encountering a change that cannot be filtered out, filtering is
- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing
CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
  * This strategy helps to minimize the overhead of performing a hash table
  * search for each record, especially when most changes are not filterable.
  */
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0

Why is this defined as 0? Some accidental residue from performance
testing different values?

======
src/test/subscription/t/001_rep_changes.pl

3.
+# Check that an unpublished change is filtered out.
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+ 'unpublished INSERT is filtered');
+
+ok($logfile =~ qr/Filtering UPDATE/,
+ 'unpublished UPDATE is filtered');
+
+ok($logfile =~ qr/Filtering DELETE/,
+ 'unpublished DELETE is filtered');
+

AFAICT these are probably getting filtered out because the entire
table is not published at all.

Should you also add different tests where you do operations on a table
that IS partially replicated but only some of the *operations* are not
published. e.g. test the different 'pubactions' of the PUBLICATION
'publish' parameter. Maybe you need different log checks to
distinguish the different causes for the filtering.

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#31Hayato Kuroda (Fujitsu)
kuroda.hayato@fujitsu.com
In reply to: Hayato Kuroda (Fujitsu) (#24)
1 attachment(s)
RE: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

03.
```
if (IsToastRelation(relation))
{
Oid real_reloid = InvalidOid;
char *toast_name =
RelationGetRelationName(relation);
/* pg_toast_ len is 9 */
char *start_ch = &toast_name[9];

real_reloid = pg_strtoint32(start_ch);
entry->relid = real_reloid;
}
```

It is bit hacky for me. How about using sscanf like attached?

I forgot to attach :-(.

Best regards,
Hayato Kuroda
FUJITSU LIMITED

Attachments:

kuroda.diffsapplication/octet-stream; name=kuroda.diffsDownload
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 6276520860..90e02aecac 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -5771,13 +5771,12 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	{
 		if (IsToastRelation(relation))
 		{
-			Oid     real_reloid = InvalidOid;
 			char   *toast_name = RelationGetRelationName(relation);
-			/* pg_toast_ len is 9 */
-			char   *start_ch = &toast_name[9];
+			int		n PG_USED_FOR_ASSERTS_ONLY;
 
-			real_reloid = pg_strtoint32(start_ch);
-			entry->relid = real_reloid;
+			n = sscanf(toast_name, "pg_toast_%u", &entry->relid);
+
+			Assert(n == 1);
 		}
 		else
 			entry->relid = RelationGetRelid(relation);
#32Ajin Cherian
itsajin@gmail.com
In reply to: Amit Kapila (#28)
1 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Fri, Feb 21, 2025 at 2:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Feb 21, 2025 at 7:57 AM Ajin Cherian <itsajin@gmail.com> wrote:

In these tests, I also see an increased performance with the patch
even when all transactions are published. I will investigate why this
happens and update.

Yes, it is important to investigate this because in the best case, it
should match with HEAD. One thing you can verify is whether the
changes processed on the server are exactly for the published table,
it shouldn't happen that it is processing both published and
unpublished changes. If the server is processing for both tables then
it is expected that the patch performs better. I think you can verify
before starting each test and after finishing each test whether the
slot is pointing at the appropriate location for the next test or
create a new slot for each with the required location.

Yes, you are right, I modified the tests to drop the slot and create a
new slot advance to current_lsn and now I see a fractionally better
performance in head code when all transactions are published.
Graph attached.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

patch-codevshead-code-modified.JPGimage/jpeg; name=patch-codevshead-code-modified.JPGDownload
����JFIF``���ExifMM*;J�iX����>�Cherian, Ajin������17��17���2025:02:25 15:24:532025:02:25 15:24:53Cherian, Ajin�� http://ns.adobe.com/xap/1.0/<?xpacket begin='���' id='W5M0MpCehiHzreSzNTczkc9d'?>
<x:xmpmeta xmlns:x="adobe:ns:meta/"><rdf:RDF xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:dc="http://purl.org/dc/elements/1.1/"/><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:xmp="http://ns.adobe.com/xap/1.0/"><xmp:CreateDate>2025-02-25T15:24:53.165</xmp:CreateDate></rdf:Description><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:dc="http://purl.org/dc/elements/1.1/"><dc:creator><rdf:Seq xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"><rdf:li>Cherian, Ajin</rdf:li></rdf:Seq>
			</dc:creator></rdf:Description></rdf:RDF></x:xmpmeta>
                                                                                                    
                                                                                                    
                                                      <?xpacket end='w'?>��C

		
'!%"."%()+,+ /3/*2'*+*��C
	
***************************************************���H"��	
���}!1AQa"q2���#B��R��$3br�	
%&'()*456789:CDEFGHIJSTUVWXYZcdefghijstuvwxyz���������������������������������������������������������������������������	
���w!1AQaq"2�B����	#3R�br�
$4�%�&'()*56789:CDEFGHIJSTUVWXYZcdefghijstuvwxyz��������������������������������������������������������������������������?�F�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��+����n�;��}�Ww��Y���QU�8$�h���M7�����/��JK��ZkY���b�/�^@ �q]�����[�L��BA���@(�f�����������{����r���n�����m1����Y�.M��v�,z�]EW����D����a�4����h���������f�EWMB�H����#p�c��� ��Z������%A���*5�����b#>`a���[_�^�P�W��
���'����Y��g�� ����O�M�!���1�����QU���������~��@�~T�1u�cu��yy�w���,QErW*�5z�J�����X%����8RN��Y��u�VF���}�-{KK	�+�Hg�b�NyS�F1�#=+B�K�-�����,r#��	���jv���w[N�� >�Ig���B�/���_��@(�������A?;��Iouow�k<s&q�7?J����e-������5���In��$O<q�]�{�����KEgG{s6���-����H�,��������V�b�?g[�s>q��F��(�����Qus���@���F�$@�"*�1`�J}��R:[\�+��*8b�\T���
����@E`x��r�[��Q��[�������c�g���[O���x�G��u&d�����c2��
�	��������Iq�x� y����~U�w��;/[��G���L�(
�X
��9�
�*�mm���Em����5���d���d!����;V}�X6H#��EAo{kv�-nb����6��)&����b���)���?��S&�+x����F�]�?�O��{�l!��]5���;��1����������s��2yV��]��ROz������X��k��%�u����n]{�H�Mv4UQv]_�"�,��#;O����9�C"������@���x�=H[�9����\6�����)<�5�swog{���I�i(?�ME69X��:�0�e9����������v���\zP�����<���
�s�=qPC�X��a���YTI#��4U-f��Z�����!8��I��PXk���� �KO�yy����
J+����$>����m��^c&������i�^[^)kK��
pLn�MEV����O�����y~h��T��Ck��d�1��`��4%�c�&��������Y�r�#d�f#���x�]����>����"��x
��'������5����l�7}����A�����t��[�[����l&p�>���)�,rF$�����S�G���s�1�������k���\I����Z���.oml�7w1@�y�?�L�����)�@EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPE!`1�Nih��(��(��(������Ul^8�N���R���	�F}���w�M��%�����g:�@I#d����Z���o���WU��S�����K*87��I%�?J�������*�M+E�����x�d�c8q�d-�9'9��dd`�0��*�
/�J(g�)#�E���o�@�Z��]f���1���Xg|S����~.��U�����H����|+�M*�=a�D�Qz�v�'&0r:�p�,�UmH@�D!irrP���y���]_�<A�h!��7���?/n�~P	��.��aZ�� ���x���o:����H�_n}��=�z>��O
������drO.<�Wd���
����.�p�--�y��}�_�Bv��:q���K��(���:rX��k��*9��@��S4�h��OW�k��v0N��r$b����������l��[��si������o����8�I�(�y<k����I zq�@m��6���Um-�^��k��Z�H�����=�sU���4����tM:K�������<~Y�px `��z;�kG����K��Q�����D�0��?(��T�<	��6��]5~�n����wu8 rOL�J��5��+X�����l��+�,j���Q��p>����>T�k���O"im�/������	
[��Q���n�K�������b��O6��'�lc<�
�<;���>�,bk�g����s��P����G�����mo��A� �RTq��+��?����-���ty'��J5g�o�	z�	������zv��
@�7e��R�Xf��j�tn'�Un>�^��I����|�����z��P����}��0y�J$e9���7���kq�=&�����O<��y�F��� ���c�QUP��V6��_���S�Y.cR���M�A�b�N�L^�b����Z�h���'p	8�df��U�������m���}N("�7�S�6>��'��g�x[D�t��6������;#19-��m'���/��N�R;���N����&�v���{��^������Q^l��4I�r�0������4��5K��qqu�Xn%;��F� V<�`c�v�U���������\8'���O��:�o���Ch�e6�2Ny�w�94���V���������������)�����B�~���A���i�Ck��hdWN���Z���a���	^�t�&�H���B��~1�����F����k�Ngfb�/o���h�|S��~���='@��-���_�@�p�
IG�k����=g�}�Z�+<1h/���p��>l��
���k�#����&sH�W''h����+x�vgY��~����M��,�b���@�������[*�B����1��OLV��<
��"���y
��I�[02���+���s��k���lf�Z�[uk���4��c�v��2k����t��nm��n��W��">��H�9�J��E�??�-�����X"��w���A�9�s��n�cof��-�x�m��.Ih�(
�c���k>�u��u�i�-��-�WdgO��R2=�[}J}6�O6QKWG�UNT�zP)�i��_6�i��E�3Wh�D{�8��+���t�mcV�O�]\J�w��`�o����\i�ww�����,`���,0Jm��ec{ywin��z����?��r?��Q'�������&���6�m�x�V��-$�Kc��1,I8
��M&�Z�����@d���NA����P�U�*F#�xw�R��=&��4
��WrB_�jK��a�����]^������-$�:}���G�I���.=H�>xK���h����G*�9;Fp���ik�u��u{�~��%�*���{
��M��~+j:��SCca�}��1��u^�����>�t�t]L��K{G��
#h�>zt�t���tMf;e��W6��	�^1�`09�jH|/�[���QY(���\B������9�<���O�X�_4!�ZEf�V������(rRz�4/S�
��<����M{<S\^]0�2���v�1�z����;�J��h]Z�Xd$���u/x{U�$�����%�H�x��������8��n���4��!g����J��j����<�"���v���x{}%#����\,nel�~=������}���\��=��<��G�GB�r�3J����^��2�!���Fm���NNI<�'�(���������F��E<���h�@[�?�uvZ�����k<mi$bE�0����Ha�x ��]��Q�w:Jv��,��1�(���2� ���+�m�,wvQ�B��>h�f���Hy��r�i�[?��>f�O���E�m�u>pON0q��'V�����o�c�_���E�0�p@�j�L�4�Km;N��G,d���~�>���<���4��S�Ao#_�,By�R��������Sjqj:���b���
PY�@m-�	�*#��v�����WY��
@G��G�p�G"2p������k��?�)5[%�hr"�X��`�A��exE��H�8�������Kmgm1�-�Q�A `3���<l�����tE��d�a�	p2}��]��������F��1b��I$�S�:u����c������#n�(��l�����f+�������\��D����\�M�"������^��n.	���lX6���@e���H�
.�a�e��Q�����z�^�KK�N�������.#R��$�!��w���%���:��xM���t�
�|=��_��i����fv����Z�k�Xol���A$!�D'�)"�
�������#��0Nc���������QG���.#�Sr���;����]v���^��z�����b�+��g-��@+��T������5��.��|/�"�[g���={�m'���|/4�2�*�Y&y
)��p(�u�mo�:��l4mN�u�hn%�����p�S]-������v����R�or�K�Mq��Gr���@��+%������qfx�7����g�y�u?h���oc�Y,�[`�7h�X�=��������^�
���Kan^��lY�������SV�����������8�\n8���H�����$�M�[K"�EbY�$�I���z�����4S"����2���om^�^��F���
��zb�T����.��k�Gw��i�L7!G�w
x'w���m�4�2�	�6���Pn�����hk>�u���;%�Kq�)�n��2�q�@k{3������#��[j���� ��?9T�,=���_�:���BM&�I[T�S�|���>�c��,�
*�E:E��1�2�kp�V�s�>��e����5���l����"� .O��q�[j:��O+��f���p��9_*"��Wi'9>��xI��M��U���%2-�L��x%�*���-^���R�s�Y�v�@����G�[�4m?B���l��n.UI9c��y&�/QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE����d�6�3�@Q���I�&
�,z���c�������5o����Y�h~�������>��������[����u,?���ynW?l88��G�
��������O�����8?�0���?�={�I������I�����%�����P��:��8\��d�����r�����juo��tQ�R�f�b/�'�%��f��'�%��f��G�v:�������.!�M�����S�]�8��<����&�����?�O(���_0'������9����'�%��f��'�%��f�l|e�]h�Z��q]����iF;C��,?xTr����X���-�v���$$+����@����{�I������I�����7$�����-�y��Y�&dB������pOqY���--�[+&O&����}�I��|m,=3@�jO�K�~�#\�S�K��y�����5�����l�7���$�M���,�@'#�9$S\m2�?�!�;{�$)v��<�r�7����4c�Q��q���)�jO�K�~�e�x�H��	���."ib����V ��'��w������{�1����+�c��b��p��<��[�����OM�FjZ�;�����egR�:V(��(��(��(��(��(��(��(��(��(��(��(���|?������*5�.p>���v�	(��(��(��(��(��(��(��(��(��(��,I</��G"�ta���A��+��4
Mg_����-l���j�9��c��?�5����-��"�?�<�0�,q5�89����+����N�o��6�J-��%��h�Q73�\���x�A=����;�xr��]+��1��e��P�xs����W=����*F��b�����g�"B���t4������>�������N*J(���N�k��j^�Z�gJ�fp��8^A�xZ������4�
�~��V��o�mb�L.%2H����$q��w��^����z�������B��L��c8�S�����u/^�jE��4�*H�W�h��`	f
�7^�����������N{i�~�us�$�?�����}���c$s��:�(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(�� ��������� ��wI#��������Z�������8�������+���t_���1f��N�J�1�����7��(���G��W��j�w����H&W)�9��Z��E������[9����m�����n���N���x'���t��<�q���;y�1��OM6����;]0����0'�c��=���z��DMhh������-L����ns�s�/�I��z���gkwq�*	�Ug���'��z��J�������i�y�����H�����*��l������^{/Ld��U�������u^1�Y���������x��,~����}'��$�a
����$��k�9$dg�����;�[n�r��i-���'��y5��k����kP�#���x���E�
��������5�I�o������0db�>�/�i�����Av��I���=�����U�_�O����|���)��	��<��}k�a@��E�o��J���1w`�>�/$��k��A�=+�
1����y]L�I�p�����LK?
[��
KT���y.�I!f�-X�q����X�_�ol�{K������u��U��>` 3���A���t��6��2���%����t������-J�[ki��kt���)pFF��t���|=����K�Mw�E%�/�X�'$�*�A�*�����Xif�e�����A���?L�m��qs�ss�������u������smo,����KqdB��Xg q��9���+U��i���J����y�0UR��g&�����Esj4�us����i���=:����t��m��E�����C1�y8Q�;�m��Ofne/h��]�����P}gJ�����t�b����U��#�N9��T�����,�?2�_Z��!��M�����T���n��������T�+�~p2y5
��g{�������&�&`�C(`1����s���t��u��\Ik%����������,��R��mm`��������[h1�7.C����u�(����
����%U@�*(�a�V��vG���^�����������T{e�������:6���������IEFV^q"�|��zR�d��s����)�d�.:�w� Yx��}~^��IEG�^?x�9�{�tm��z/O����J*2��������V^q"����	(�m�?���jM����������	(����d^��h�/��^������T{e�"�����������	(������/j]���E�����*0������l�~�zs����J*=���E�����e�/N>Z��_��~�q~�Me�(�����j�/�S���&�'��/�O<�������/�^�$���/��^���F��������$��+/8�}�Z]��}z�w�>��l��X����������	(����E������E�����IEFV^q"��F�y���|�%�s��z�w�.�x���?/j}Yx��������/��^�������>=����_�WzN�k��+V�7,w c����kl��X�?�����>.�c�m���d��t����(RN.��neRt��g�4��|x��|����'�����?��I��2���p��f6'���*7C�#n�������V�s���#��A��j��J�Dn������l�3�Njo����i��0�7��KG
�Q������D�����Q�t�x��~T}j����1�z���~�
����9	��?I)���[����h�[�>Q�r��zo��|?������J��bn����j>�[������=/�~$�%�;G�4UX$a�S��|x��|����'��7
�����j7C��m���J>�[������=:����'��+�4��� �7m�8���v?���5�>�����x-�Bn�\�F1&j=�����>��ieh������P����.�qj�v��������������5OR���=V� ���1,�)P�*��<�2�����M����*7��<����~T��[�?B�T�����{6��C�~ �5�Ym� ,�c��G��o	�j���_��8
�:��s�Os���4<f&�����F�x��t����Q�����Yg���=���W�u�����G����6�����A��|���;���]��mL��'Kg��RA�`��y����?��q����(-8��c���Q�����9g���=�������������������������/8����������n�����������R�?�_y��<w�x������@���;�c�����_�&��$l8���y+�
��$Q������*������w�������ME[4�%��g��_���,��O<�������H���Or)h��(��(��(��(��(��(��(��( A�������-l������D�UA9��OEP����*i&�4�[I%�X�B�[�$
-�-&�Q{�]6��2^t�C�z���j�A�%�A�>�ho���0�������y�i:��ww�m���X�,��2��`��W��k�?�<?�a�?�V����/�X^1�Y��������%����������>v��K��o��4�*�3)���?Z��������?�������g����h����g����k������h��0J������;Y�&�|7��I�(T�Y���~��z� ��|��������<��������������<\�9�f��K������c���������t�W���7����^U�>a�V]��u)e�]���I�[���[f12�*����5�^t��������t��������t�)��Z.�s�G"��"���[p���;������5��M����$��R��xY� ���e\c��Wu�M�>��}����M�>��}���F�iv������������0�M�2� d��j�����i����=��4me#F\3�&m���#5�������k�4y�����k�4���5+h��n�u(�L��+p�~�!������"�g*����mlc�MC�����h����N��ReRG=���G����g����h3��F����j�U�1���j�s�8��Z�g��"�����:o����������]��������0bK��Vh��(��(��(��(��(��(��(��(��(��(��(�������/�5%0������N���T�(��
(��
(��
(��
(��
(��
(��
(��
������G���R����O��)��������;1�
��3�����Fp�QEx��Q@Q@Q@�
�Q�g,8'�S����<����t�i�=��|/�v���?�
(����QEQEQE����������^�^/�?������������x��3�>)���_�w�IAEWi�AEPEPEPEPEPEPEPEPEPEPEPEP5����������k�?�<?�a�?�WK@Q@Q@Q@Q@Q@#}������JH�!A�>Q��M��S�|��uQEQEQEQEQEQEQEQEQEQEQEQE�p~�����*5�.q��}�IEP]��ow�Imiou:D���Y�]~���Ic�������V2M4Wp�XI�]m���x��q��FGQ��r�/�u
#R��+;���R������S[9�����r�c�	�vq�Z������so�Og$�u��s�c�O���
����l�.&���������i!��3�N��d��t�u�+�(� ����
�X�$�\���<td��,M���+�+g#�}��@<b�MR�L������O�k��O �~R|�L��H��G+�(����5cp��=���h.k�##����g�6�\j��k�{K�Q��v!p��I��Z����(��(��(��(��~0��O����E-}-_4�a��������Z����_�<�����gEW�|�QEQEQE-���g |���)�%���g;~a��GC������lO���0��)�Q@Q@Q@��?�������������������^�_I��w���?��������QEv�4QEQEQEQEQEQEQEQEQEVW������/�i8���V8��f�=7��Zi����k_o��������B��6nY{g���4W
c�k�>��'�.��-��Yf�6�Q��4l'r��2y���o�<@��\�nm�N��eM��	���wdw��z����^���c��U��mN�`��Y|�39
��1�T���<Aqq�+�
��O}�X7���!+��Q����������f������Z�M�:���m.�k>H'j����]E��i}h�6�����m#<���j�g������H'��
��phJ*?>3���#�G���g��@QQ����t�h3�3����@QL����|t=i<�������@QQ����u���O�����������������ix�7��=
>/�)����)�sF"\�@�4�:<�w|t=h�S<��>n�
 �3�7^����������������3���Tfx�r�:�iL��An�Z}�4d�7|t=i��a�����T~|||�Fz<����=J�����[����g�g-����	(�y������I������phJ*?>3���OCG���g��	(�����v�CG�������T~||��:_:?�w�C��EF'��
���>3����C@QQ���{�z�y����O�$���4�j�?�#�`�;X�������[������ym_I�����c�����_���-o�jw�����)�e���K�}�>�s�?��f�m���<�Sm(�W�^E|��~�z^B����~%���l������I�is����u������8�:���y��������4}���~%���4[Jq���r(�<�|�Fz��{�z^C��s�?��f��\���������P	��n�:Pm����y{�z^C��s�?��f���I>�I9&�,��������9^�����t)Q|9��gx�`W����W??�gJ�/w�#N����[�����g����^���%�~�|t=i|��>n�
>��O��N
|�����I_4�a��������Z�K��{�z����//����r"�?��k������sd�n��ptT��R�dr:Pm�;{n�:W�f|�+"��6���zu�Q�iA?/C���E�r��TT�f�����iE��^��E�r��4T��S�/PH�Q�yq������E�r��lqu7}��FM2�[[J.��'�j3m(�W�3��CN����>'�Oz?�#���4������f��|uiY��{Hw"����az���}�^>^�#���0���GEI�yq���u(6���zc<��Y���r:*Sm(�W���R}�_���Q��0���{����_�v��jW�W����_��y+�����u���`������x����k���J$���G���Z��a��t5�|��)��e�Z(��(��(��(��(��(��(��(����&��"����+�b�"���������|!�x�i����L�N�l�1q(��1��]��Xx_Y��4��M-�A�[Km���tdl���yn���
i���w�i:�5����g��Xt]�������(������}9��q|n�R[�U2y���-�=J�U���nu�}{8�u�>��n�l�1��e@<��j(��%�v~���U��mg�T�����k�?�<?�a�?�WK@Q@Q@Q@Q@Q@#���R�7�?N������;b�M��Ju���:�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
������������p����!}��I��
�������O���7��o�,�:(��l��(��(��(��+���E�3��"��|�_^h�-i�����+��>)�q������QE{g�!EPEP_4�a��������Z�Z�i���%?Q�r���f?�^��y9������(��p��(��(��([a���3�3��4�}�71���p)�=��|/�v���?�
(����QEQEQE����������^�^/�?������������x��3�>)���_�w�IAEWi�AEPEPEPEPEPEPEPEPEPEPEPEP5����������k�?�<?�a�?�WK@Q@Q@Q@Q@Q@#}���KM������"�R��v��i��
�����������p���~����������u����{���F�����<L5c��4#0��O3����y�������$�&�t�X��ufy�g�X���w`�! d��\�����[��n��dIn"
�����T"���t���}Q�23��uRH��y�[�����0�������O�^��[-h�r[H,�����L@�����6�4�j:�����M�������#W;J��R8���y��<#���ZE����!X����d������|Q�����C���@��H�J��>��N(����{5��/�;]wZ����5�	�cY���1F�,�&Vo�/=W�9���SP���MF?����,��Fb:���8�����/���7S��#�������{5��/�=B����_>��n��|G�����>&i^3�����K�e�9i�B�W1��Q������2���4)��i5�;*(�� (��(��(��(��(��(��(��+��?�_F���B���?�����o��'��+���/��}?
�������������������(��(��(��4�������@�}y������_��[,��~y�����$hQE����Q@Q@|����J~���?�)k�j������G���R����z������^��8z(�����(��(��(�	m�����
��t�S�����|��3���C������lO���0��)�Q@Q@Q@��?�������������������^�_I��w���?��������QEv�4QEQEQEQEQEQEQEQEQEU]OQ��4��B��m��2HUw�z�����V�i#�����K�f�v���hb����c����e��Q+Fe�MG�F�������h�����3����aa�����������'�4X���<�o�ay+)��Lm
��5_��6��w��0�;�,�n��.}�������`�M]-s^0!���NA�  ��5t�QEQEQEQEQES'��y?�?��L��=��t�*�>=������uo�j
���B�����+����m?�zQH���(��(�L��#����_�F�^g^���G��������p�����g��+��}EW���Q@Q@Q@Q@Q@Q@Q@x�������O�W�W��~�����\$���qc��������_�������yQ_6~�QEQEQE����"���^���C��4�������@�e���8��T}_��
(������(��(��~0��O����E-}-_4�a��������Z����_�<�����gEW�|�QEQEQE-���x$|��2*eIk��{������~����'�Q��QE����(��(��(�d����������J������n�������h�����������O�������J
(��O�
(��
(��
(��
(��
(��
(��
(��
(��
(��3�C����m�������R�6)�wy���^Z������z�]
rK���f �A"��	�{`z��:,�U�2��d���X��Xc�3���Q���8�}��u�	E���il����1��V�cT���py�O�T�/�u�D�u�������$��Q���U��^���z�V�����fC��(����m�I�d�5��@�3}Oz������z��T���Zp����-q�\/R6���U�OP�������������M�����4T[�x�^�m�7���
�M�p��C��$�,���G���C��8�Y�E��	Y����w�q��C�p����]}���6����ez_(.�}���<c�������5t����N����)I�e�<��jJ(0�q�{s���d�����{��P{$�����AI9����zT�P6I�=;�w�&�8������(��I��>�/ZI�^���(��I�={w�2t���~����T�����O�O��ks���_���D����5�����c�w����B�����5|�z��T���vH^.qa���)�.q9��TTR�|��l����x�����I�E�������*:(�r/��Q$C�>�7Z@�3x����Q�E��M��c�}����__�W�R��.~��������|���������b��+��}O?�Y]oN��l������$�����������,����^�|�()'8��v����L��������d�~��?-IEG�N?{��4l��z���IEG�O�����l����c��RQ@����w��.�?��|�����(0�q�s����d�~���jJ(=��^����F�9��n>Z�����8��/J�?�?&������m���W�W��~�����\$���qc��������]sj_�����'2Ds�q�7JS$\�z|�**+�.~�����<�����~�oJ_2.?s�?{��EEE�6H$�c0��~n�o���v����Q�E��M�o��=�����d����c��Q�E��M�"�9��������4��`8�6��|�_^h�-i�����+���#��7�T����
I�%�?���s����������"=�g�g|�����$�����O��#	'�>�/Z6I�={w�IEG�L���������E��~=�$Y;���-}%_4�a��������Z����_�<������3�1g������X�.����j*+���7d�H�q=>n�y�s���|�EE�_2/���?{��H��=����TQp�d�H��9������������j*(�]���3r�@I$�0s��������>n�Z�uw�8'��oC��W6~���'������������x�����QQJ�����l�I��?7Z7���{s�u>�\9��&�����>�Z�s�q�7J��.��l��8��7JO2/���?{��GEE�6{'�B���� ��y+�
Hs�q��������������J���L��~���+f����%�'���������q�3��i�Wa�b(!@c�����(��(��(��(��(��(��(��(Q����=�����1�^��!�@����=��Z��!���]�PG1���:���6�u`��z���L���:u������W����y"��c�278��
h����[�_�t�}d�j&Y���UA8]���x���EQEQE�x��g���3�j�k�������f����EPEPEPEPEPL��=��t�*}2������������B�����*{��\�V�f���{�����QE"��(��(��3�?��W��
�y�zg���?���6*���>���������Q_P~QEQEQM�X�����8�ngc����u��	o��;���R����w���o�'��s��������4I�-a[�������R�I��C�P��_.����Z�����qM$e�Xbb����s�
U�,�������+����)R��x� ��3��VU���[�	m��5�^y0E$�Wm�##�������p��������x���q�Z��N���O�9	������������0����]�FU���,�Nq�RzzV>����1�0��C�Jw����u�U �	ld������j����-�V����Z��m�I�wl���E@?Z9��=�;�����Z������X/$������;�8���tU�E�h_��\j�!��>����.�(2Y���8��?�-���t��
O�����C�5���?�_F���B�g����wM�����������i-�����C c�����+�(�
'��>��'��I�7���.��+�O�B�(��(��(����?�Z�?��/�W�u����"���^���l�����
�����EW�~bQEQE�O��)�������������?�S��!��K^vc���g��z��������
(��
(��
(��%���<�?0��?e>���2Hp��@�P�?q�����(�L(��G��EPEPEP�~���u��m���{Ex�����_�v��jW�W�`���������F����%Q]��Q@Q@Q@Q@Q@Q@Q@Q@��+�k�X��^[P�e��gY���~�6a��1��<�|&�0��:^��YK����%�B3�y�F~���%X|Cq��x��K�l.V�w����FF�2���Gz]:��H�����l�	���,��Xef���#���u4QEQEQEs^1�Y�������Z��c�������5t�QEQEQEQEQES'��y?�?��L��=��t�*�>=������uo�j
���B�����+����m?�zQH���(��(�L��#����_�F�^g^���G��������p�����g��+��}EW���Q@Q@d���EW��$��MkVO���Q���O��O�dT���=Q_$| QEQEQET������#c������y����7������F}��AG�_�K#��*O�p��(��(��(��4�������@�}y������_��[,��~y�����$hQE����Q@Q@|����J~���?�)k�j������G���R����z������^��8z(�����(��(��(�	�N���q���Tu%��}G���w�GC������lO���0��)�Q@Q@Q@��?�������������������^�_I��w���?��������QEv�4QEQEQEQEQEQEQEQEgj����V-oL��D������=pH���h���>�4=&����5�*����'���[�q���l�����7�����F�<�mo}�w��q4����]irl�Y��a�zg&�6�-F�I�g��m����$��������Sx��X����IMF,
���B#���*�}s��;�>�T�{MF�;�w���]�pr8��9��=[U�F�jVB��H��\0b=�zv���
[������R}JV�ss��d����M�`�ps������k��W_��f���Q�Z2��u,�026|���5���6�J��[����L�{[(���hp�C��r�#h��5��i�������w�i4�6��k�� y7�0��9�cK�k���)]Re��D�!W�0��QS�r@'9�S�Z����[�i���+�|��"�Gd�ec&A$1;x#���,��Y���E����e�l��v��]�!q�s�8���
�������=mQo��������B�6�U�G4_���M�x[V�$	6�k;$Q��7)8
�������Q���n`K�t�Iu�*���X*�;X�7o��-Z�Ge���<��~����)�q��q�������#)����m�y���3�������#���$�g�[�e����m�%�9}�>��'��p�q�������#���������Q�)��3�������#���N��V��L�e�<�?�?���>�=2����������7��c�G#2/�����ww��������\�V�f���v��MO�k���C�����4���~h�#�Z���TR��O�����������j��'�� n������5h���	(�s��>}e_��9�#<����TtQ�Z}��$����������+�b|uy����s����H�3�^c^���G��������p����<\���.��O����r���3�����9}����T�W������g���6�
S2����o���@��=�>��'��p���o��(���8�����o���H��=Q@`Z�LnB�=�J�����������	������?C��n��������4w8���?������KC��%�;�|��3������>�}3�������@�������*����~h�8�Z��QQF��/��?z>��Z����-���������5h�w8�G��2���)�����se\t_~��j�S���Yg=xZzX�~���v�������Z>x�O��6�3�G��5_�����������I>���������_��4|��_����@����I��a��=e_���~9������5h���	'���������?���~h����_����@����I
������j��k�]
U_i���������F��4�������@�e������{*7��$\3(���m�y����3o��(�h��g����q�����9x��8����(12�p���6�
<�������T�P~r�������
����M'���R�2E����^��+�:������G���R����z������
����@ ���2(���?���b��_j���t>cBSn�9h���O����3�G�������4��g�G��W�Z��|��q������BQn�4|��"��C�9������=}�*(�4-[[����8 ���z�f��~h����_���n�����8�GC����1���<���2O���������4�gs�Q�������Z�Z}��$���>h����_������#?�W�j:(�->���O!����������n�?4|c�Z/��tQ�Z}��$���~h�8�Z��I�w��}q�������@����Od�<�����A���zf����<?�����o�~�[���o��+�+�0?����g��Q�z������y���q��()����o���]����A�����(��(��(��(��(��(��(����}4����MyobojX��c�kd��0��d�@&�|#=����?_jw�H���^��"M��#I���@8��Y�i�E�����f����P'�F�A��$g�����n|7�X��O�Xx������W�������&5 ������QEQEQE�x��g���3�j�k�������f����EPEPEPEPEPL��=��t�*}2������������B�����*{��\�V�f���{�����QE"��(��(��3�?��W��
�y�zg���?���6*���>���������Q_P~QEQE����u_����A5�Y>*��GU��I?�Q?��S�~���QE|���EPEPEPS�����3�z/������T_������^�C�x�J
>���YQRKQ@Q@Q@}y������_�������E�3��"��z�g�#��8�W�#B�(�l��(��(��+���?�S��!��K_KW�?���?�C�������+���'6�����E�.QEQEQEKl7\�S�1��O�8���G�>�O��C������lO���0��)�Q@Q@Q@��?�������������������^�_I��w���?��������QEv�4QEQEQEQEQEQEQEQEq�m���l��B�uFK�*	/a���r[�������'�R��K-�
B�/`���1d?�q��u���Z�6�e��������N8�����72��0P�|��z���]Jq��_Q��t��u���������QEQEQE�x��g���3�j�k�������f����EPEPEPEPEPL��=��t�*}2������������B�����*{��\�V�f���{�����QE"��(��(��3�?��W��
�y�zg���?���6*���>���������Q_P~QEQE����u_����A5�Y>*��GU��I?�Q?��S�~���QE|���EPEPEPS����-��N���P����F2q�7������q�_�PQ����������\(��(��(��+���E�3��"��|�_^h�-i�����+��>)�q������QE{g�!EPEP_4�a��������Z�Z�i���%?Q�r���f?�^��y9������(��p��(��(��(kS���m�0��GO�8����r�'��P�?q�����(�L(��G��EPEPEP�~���u��m���{Ex�����_�v��jW�W�`���������F����%Q]��Q@Q@Q@Q@Q@Q@Q@Q@�%����c}KW�I���)l�$s�V������^�:{Zov������3�lT����A�+�+D�c��t�����*\K,�	
��B�����z����ZF�t-{P�U���k��X��de_����t�w��[�����
ov���]7�V7�\Ayk}��o	�x�Dd�:� �@��Gq[��i�]�3$��dFr��W���:��!�������<
{z� �R>Hc����r����
��n�~���{i$��M���]D1���p������x�J��8m ��"�v���X
�t����C�q�k���/k��E�um*��_����WxDk��s�E�U���S���{x'�W�n��P��e�2�����?H<���Oiqj���x`Y����?��bq���\��c<f�W����j3Z�C{2�������x���#v�����W5������sn��<�A��Z6��&
�m�<����G���M"�)����>�@�o7��{T�G��A�A�J��^�'�#��[X� ��j�����q�|)�=����ZB����2����n��Yee5��A.�uv�Ng�G��'�����r���o���s�.��c3�p{����	(��O���8���G����N����J����~'Lt^=�Pbs�O �:���	(������u�E��Q�>G�������J���D�~�C��^JN?����:/���$�O����'����N����Jd�?�����s�}>�
n|�}�!������r�T��o��	%���Z��L��w�u������k]���r|��{y�)���un=��fC�[D2F0[��z�������"�����}�����}�<���F��q�����g����2**Q2f�#��-��=@������-��^�[�9��������>���y�`����W�P����q���z��|
q/��6F��������-#�����*��}O?�����O/�=����|����~�yO�=��������?$��19������A�� ��x�(J*?-���N����G��~�N���P�����u_����A5��?���������te���ZWp-$��?)�3�Y>�|�EL&N?��<���/�@�?��8�v��������]�h�����F������Z<���{��q��>�h�Yw!��3!�-��B�~����!��������ae�����L���_{=[���( #�#�OV���Qo0��CS�����7������2f�#��Ks���U�����gs`�/8�Zv��~�?�������)�qo�����A���#���M���>g����2**_93��u�2��Z<���.����}�-���_��dTT�d��G�[���98�G��:�>�h��s?����QR�������r�^�h3!��4C8��~�[�9����������E�3��"��|�fC��h� �o��}i�F���4�]A����Q^�[�H����t��[W�#N����?��p;/�S���9�^?J����(������:���](����I�=�n�%��?!�{/?�S��y:c��}(J������G���R��^S��|�1�>�����>&_��$*�e��[�K�~��f?�^��y9�����0����c$�>�i�G�{�>\g-���Z�l|��r**c2��!��-��=G��?��rs�����[�,���Sy��>�}��o��t�M��4G=[�o�E����
0�3m�=Ks���&?��?��������Q`���Q�������6���@-c$������FfC��h�H��~���s��]���+���QR���>�u�V���d��h�9���������3�W��(�?��8�v����L�h�������a�����EEK�&?��?����>�h3!�-��n?Z-���_��dTT�d9�F�d������y��>�}��o���g����3��g��������R���c�!�_^'r���{5�f'9����������������C�oW����Q%���\�s�.����<����t����(�PU@,X��d�T�QEQEQEQEQEQEQE�j��5k�G�����D����V�0�?J�����h%��up"1)��iB� �����(��(��(��(�������f����5�����������(��(��(��(��(��?�{����T�d���'���P5������.?��3PT���������A_�?�i��(��E�Q@Q@zg���?���6*�:�����=^�5��lU���<}O?��]C�
(����$(��(��+'�_�(����'��kZ�|U�"���^��&�"��������#��(��(��(��#�3�����/j��8�$w;�>������}��AG�_�K#��*O�`��(��(��(��4�������@�}y������_��[,��~y�����$hQE����Q@Q@|����J~���?�)k�j������G���R����z������^��8z(�����(��(��(�	m����~a�8�i��l}�=�q�gwO��C������lO���0��)�Q@Q@Q@��?�������������������^�_I��w���?��������QEv�4QEQEQEQEQEQEQEQEQEQEQEQEs^1�Y�������Z��c�������5t�QEQEQEQEQES&��y?�?��L��=��t�*�>=������uo�j
���B�����+����m?�zQH���(��(�L��#����_�F�^g^���G��������p�����g��+��}EW���Q@Q@d���EW��$��MkVO���Q���O��O�dT���=Q_$| QEQEQET��=#�g{q�z-CS�����n1����g�p��}%���:(����
(��
(��
(��
��@��kL��H��_!�����Zg�zE��
����G��q�*>��F�Q^���QEQEW�?���?�C�������~0��O����E-y���W���Nm���3���+�>\(��(��(�����>�����L��]G���s������>��b���QH�x(��(��(���O���n�������h�����������J���L��~��������������+�����(��(��(��(��(��(��(��(��(��j�|�����J�L����a����6�>�7��|?{{��:c����.QoY���'�����=*��������%�������m����W�b�D��!�q�U�g�lt�	��]Z���Z��;�%�f���$*�N���������=>������������~�����v�r��B�y~_��� �3����m}�M��E5mB�M������ YK:�M�pC1@��x��0&����js]xo�����y�����	P�Y	��@����m/&��Y���-]��eh�/=���{��)�cg�S��Q���1��\s��?Z���H�~�N}����1�����U����T~cs��:g��]k:���n�=�:t�i{&8�����C��5h�KiZ���+F��F�&�m�,����#��-|S���s%��hmAif8��x+����5���/i:���V�Ja��^�ByC�@#��S�����n�X	$E8;�a��<�=(B�?�{����T�c�:g��]i���g��/�q���ks���\�V�f��w�)������7��PyI������
�t�_ ���U9�E����R����p���t�0���DgvnJV/�I�QRy)�?��F��Q�&G��������E��I��R�P�7������)3�8�n=�Q`�_�du����z���k������J��g��
�t�J��?^lu��%�>L��H���k�
�S����W[����T~c`~��g�����1�����U�������QQ����q�����F��.y����)�cg�K��Q�������_Q��@VO���Q���O�����~���~����-�=P���%����CS?��S�~��TT�����8���JI����f�������dTT�R`~�>���{t��L���z7����)��:����N�#�����J,�����O����:7���
q����F����fES�����X�v���$8�������*S
��3�/�c�X�������>������)�,�EHbA�O�����JaA����=�������:��"����?��\to��/���G���E��I�QR��3�3����(������=��E��I��RyI����g�s���s���1���(�s��2:��@��kL��H��_#��g���@���J��Bv_i�D��\�s�z����G��o$�Q�w�#N�����X��w�G����������QQ����/�G����n?t�q�q���EF$c���������7����z����$��~0��O����E-}%�6?��L�_��|��v5����H�H������;1�
��3����/_�8J*Q�3<c#8!���A�q��3�z7���3fEEJaA�\Fq����%9�H������J,�����O����:7���B��F2}����
(�?�=Cq�Q�&?���=��E��[_����>a������+����NF��:�f$����c�s�P�����\>&�����R�)�?��F��Q����G���J�����L���D�~�1��n?J<�����3����E��I��RyI�����F��PbA���q����E��I��R�S����:7?�'����\to��u�&{����_�v��jW�W��[�����'�+��3��8�^Z�L��~���;�����%QL�?���:���F8���q�{������H���T��=�h��(��(��(��(��(��(��(��(��b���se��M<e��{�����#K��H�P��Lxj�D�����	''9�����kJ�\:~�guu�^j'O�k{(�vm������
�O�%���������N�������9�!w���I�u4P//�m����/N��^I���[I��v�
Xd��4�[�0j�����W�P���onSm�Q��RT�IR2+���9��xrT$q����:+��k�?�<?�a�?�WK@Q@��e�xcT�a����sB� ����yMy��-��l��4?�Ep��S�7���[2�F��$��?��r�=S��Z�F��$"3@��S~���q�^�k���?�
�G�-��l��4?�Eg��K��?���^�g���:e�����GK��v��U�R#E9#?�vo�-d���@�m4����N�XoQ���9��������m�A�������������C��Q��K��?�?��v����.�}{o�=������v��.yn��U�*������V[�g�q-��p�>�%�m������3��^�k���?�
�G�-��l��4?�E�T�?����^�g����2��������o�[^6�����h��Q�c��B��Hn�4<���h������_g�����B�����5Ip���f����}j:�^��%?�zQH���(��(�L��#����_�F�^g^���G��������p�����g��+��}EW���Q@Q@d���EW��$��MkVO���Q���O��O�dT���=Q_$| QEQEQET���q����������D��~�s������F}��AG�_�K#��*O�`��(��(��(��4�������@�}y������_��[,��~y�����$hQE����Q@Q@|����J~���?�)k�j������G���R����z������^��8z(�����(��(��(�	m�n����3��)��7Q����P�?q�����(�L(��G��EPEPEP�~���u��m���{Ex�����_�v��jW�W�`���������F����%Q]��Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���x��0&����������`�M]-QExg���
������$�[���!�#����B���X���������@��+��
(��
(��
r�^�GZm98�~����7���>���:�i��!�r��u��I�d��}QEQ@Q@zg���?���6*�:�����=^�5��lU���<}O?��]C�
(����$(��(��+'�_�(����'��kZ�|U�"���^��&�"��������#��(��(��(��?���w;���O��ES���I�8Z}��/�((�K�IdtQEI�,QEQEQE����"���^���C��4�������@�e���8��T}_��
(������(��(��~0��O����E-}-_4�a��������Z����_�<�����gEW�|�QEQEQE-��|����{��L���Q�q��8�P�?q�����(�L(��G��EPEPEP�~���u��m���{Ex�����_�v��jW�W�`���������F����%Q]��Q@Q@Q@Q@Q@Q@Q@Q@Q@
��mg,��%�F��1cs�A�~������z��g.��]��]G;�������{g�tTW1�x�5-J���J����
sb�%
���-�X�`���L������.��O�eh�5	Jl�e����A
J�q�(���I|yw��2��������d��l����*�[�;X��ZV�{ZU��������6��]�t��������?�<?�a�?�WK\��eI������}^���Bl����(�>?�oH��w���y%z����
������$���G���3��Q\�QEQES��b�E6����Q����1���O�z�ze>o���9���^����M/���QH�(��(��+�>���y�`����W���|���������b��/���x������@QE��!EPEPY>*��GU��I?�Z�����u_����A5�Y>�|�EW�QEQEQE1?�q�����8��
L��?��1����T�3�8�J
>���YQRK�Q@Q@Q@}y������_�������E�3��"��z�g�#��8�W�#B�(�l��(��(��+���?�S��!��K_KW�?���?�C�������+���'6�����E�.QEQEQEMjqu�0����������������~����'�Q��QE����(��(��(�d����������J������n�������h�����������O�������J
(��O�
(��
(��
(��
(��
(��
(��
(��
(��
(��"��K[Y'�;$jX�B�@��k��?3����oi}ms�]^�ln�$�]d@�`'�z��1��k>Ht��%�����0��H���jO�����������^��i���z-�{,���V#*���5��P�Koz�����u�_\j�"J!o$����g��F��r*��.�f���4��������	t��Va��a��zM�x�JX�;�]*�2�� ���jy����M���[kde�:��\�'=��?�<?�a�?�WK@�S���^���<�5%��X��i	�w<��U��h�=G\�k��?�oH��w���y%|�1�o#�1��f_�B_��1�����\H1�8�8�vqs2_�I�:c��G�$��t**(�fJn$9�s��;Pn$=H�AQQE�s2_���:������$����A���'2/�Qv8��jX��E��8��@8���D��G���t��by1�<�sL�����J�q��!������)M��9#�	�TTQvi���i��u�A�����r89�:�TQv��E��#��
Aq 8�:TtQv��
��H�m�:W�|
������s��<�H�����3�?��W��
�t�[��<\�Y]k.��y	�����"?C�MIE}9�Y�3�>����9����(�Jg�����<�����SRQ@�1��#���D��=Q�r��c��Ml�O���Q���O����dT���`��ws�P.$�q�p;�TW�]�
��~� ���A���I�dt�A������d��C���3��G�e9�rrx]�3%�D��:���@��c����TTQv��(��cp081�Ag�����0�R�#�3����z������~�?��i�,
��9#����72���z�***n��?g��i�9�����i��G=Z��.���� ��cq�p(�D�r8�:TtQv��I��0FG#oA��q!�#�g�Q�E�rC�!���#�=j��B���k���<��+�j��@��kL��H��^�X�4����)R�e��H�`C�����C�=y5%���S=\�=h�S�=i�Pb�<{�<��Lu5%���zc���>.��|M�T H������u�O��)��������;1�
��3���T�������A��q!GM�J���n���d��C���^i������TTQv��/�$�s�u�\�1���p**(�g��\H1�8
>�&1��oA������e�i�k�r@ 	����<�q�jKa��0@9a�8�2�����1JX|M���d�i��s�u��2����:�TR�?\�p�H.$�q�p;���8�p08]���bO�I�dt��t��Hs�9�<
��.���M��<�NO����:���QQE�{8v=��!3���r����d0Fs�y�������������J���L��~���$�z�w�IC<��C�={�!A���Z}�|���T�)h��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��9�������`�M]-s^1�Y�������Z(��������?����!^I^����CzG�{���+�+�������������EW!�QEQE��X�:���r�_��AQ���{����N��|������>��Os�&������(��(��(��+�>���y�`����W���|���������b��/���x������@QE��!EPEPY>*��GU��I?�Z�����u_����A5�Y>�|�EW�QEQEQE1��D��}����u
LO�$|����q���}��AG�_�K#��*O�`��(��(��(��4�������@�}y������_��[,��~y�����$hQE����Q@Q@|����J~���?�)k�j������G���R����z������^��8z(�����(��(��(�	m���<����t�S��.���rz
ec����?�����(�~�QEQEQE{'����_�v��jW�W�����u��m���{E}&��??��O��oW����PQE�|�QEQEQEQEQEQEQEQEQEQEQEQE�x��g���3�j�k�������f����EP�|~�����^���
�J������?����!^I_5��<���?�g�t
(����(��(�'����m9?�/�t�������s����g�e>o�����=z��P�?�i>�(��F�EPEP^���G����������3�?��W��
�t��S����W_���(��?	
(��
(��
��W��:��zI��������������	������?C�z(��H�@��(��(��(���D�g�Z7����jb�������F}��AG�_�K#��*O�p��(��(��(��4�������@�}y������_��[,��~y�����$hQE����Q@Q@|����J~���?�)k�j������G���R����z������^��8z(�����(��(��(�	m�.��0����T��Q�����������������G�aER?^
(��
(��
(��=��~�[���o��+�+��g��������R��������~'�?�7��n��((���>h(��(��(��(��(��(��(��(��(��(��(��+��[Q�7�5+����G�i��2}���Y9c��������j�	��{������;�T��	v��Q�������(������_��������{k�`WH���<u5lx�]�>�������8�#P��P8���9��H5�����*�*��H5�����*�*������W��WmgvZ4�uPJu\����4�j7���m/$�=��\���0�E0?���;@���
x����������m�da5�y$���\W�*O�!?�%+���������+��]]}���)r���������8��kO���y���D�[2��t9�*`������
L
�s���;������R��T�6��B�JW��_�,��
L�_k��������8���q�p0���u��/��������#����@@��2�0=J��G�(v3���v�O��I�o�'����<m�@�����l�<k}��i�N����-���cv"VGV8S�|���$wK��t�t]>�{c�5�����������c���P���n������<m�@������'��������'����|�GQn'�d�R9{�TM�q�T�?�����������4��z�%��8��Kmc�����
��3
��</�'�����������HJ��R��/x��&����n,������1���a���n@���p�x�Q�i�i�}����b(n.���L���+��I�z>�C�fa�~'�'�?��>���r~���w�*O�!?�%+���g[<:,�_��6�5����0R�+���������X�W�jA�P�������1n��@�`���c�P��w+�"�S_r<c�'������������R���/��<�i��!V��\#���1���z�����<W������O�N��Q����f���H����P����m���(���T�6��B�JQ�
����O�	J���f���P�����ee������������J���z�~6�a�����u����B$�;��A�01�9"��P�������:��Q���<m�@������'�������Rx��z��ye$��]h��
��r�����Xf���q8�t+M;O[���&������F9S�?�~t}B�o�?�������"��I�o�'��u�|'����}F�@k���h�sA.��O�&�+�k��5};L�QT�m�
�]��&/.����q�T�������o�jvb�V��m�(�>�~l�`��P������G>'�3UQ�%�-�����\��J���������\��J�������?���������I�U�k�E��;d- `�s�S�T��b�e�Rhd�R�!��_������I�� �?�R������?� �?�R���������s�
s��+��
�����
s��+��
������_���1��<�E6�7>Te��RBWyQ��(�H5�����*�*�k:���h��QxV���Y���$c��o���/�i��pl�M�i!�������Q�>��?��5��\��A<�I,i|�*vcr�ml��M]XM&����I�o�'����<m�@�����DMK�Z����`i%���I�&]@��+/b���N1]��,�.t�g�����}���q�����q}B�c����v�O�?�Rx���	��)G�*O�!?�%+��wX����������7�%n.R(� !$�2��?J��> �SjV�h�R����vD[y����}B�`���v�O��I�o�'����<m�@�����j�<oi�kV�w���-(����k�h��AS�<t8�)�:yuX,�]:�-�Wa��V�zI��V�p9�G4}B�`���v�O��I�o�'����<m�@�����k�W����
�YZ�P��
�"LGS�2>b1��{U����j���ZW���m��Ir �$�>X�V%�q�`rh����������
����O�	Jh�G��31���P��8<���~U��������u�6�o���6�0k�4[��`�#�>Ps�}3�2�dh��V�6��j1M2Eqt��7�;������?���zlVY��+
��{u�X���T�6��B�JQ�
����O�	J�q�������4��w�XKf�*y�RX��9�pO�x���Y������5���k�"Y��i;}�K�;~'�������/�
����O�	J?�Rx���	��)_E���z��'����B���N�;0p�+�u?��7������,�3
���e\&n�����>�C���nk���G�������R��T�6��B�JW���O#��!�������4^n<�������������u�ox�-?K�U����-�DK
����,��8rG}B�o�?�������#��I�o�'����<m�@�����`���o����y��?�d�H���{���=��x��d���3MK��M����PX	
�������z��
��[s_�_�
<w�'��������n��Yi6���K�h H���J���F��Z���5��4������s]�!���9c��:��Q��i�i�Cqqyd����[T�&�r�� ��������:�fa�b�(�8��.�%���	��B���C��Q�	��B���C��Uo�k�����m}w�E$�	n%�cm������Q�s��5g����������_����`Z0���tA/�$��
W��G�$��
W��]��A���w��P��T�A���w��P��U��_����xj�R�������� P�H5�����*�*���>���]u���F�T$��p��{j�t}O[������~��u�2��[��W�pa���e��5s������-����H[�V���#��R���Jc�3
�!^<�����T�6��B�JQ�
����O�	J�O�1�:~�������%����2�����T����6���#]��X,�o��nU.5��^5p����s�+��;���n������<m�@������'�������7��C�]���-�H�5������+�F���28<�i�=��M:��@{]6������Ve�$So�%q�����P���n�����<m�@������'�������ZG����v���L�Z[�Km$�|�e(H-����G6pA�V�����Z�"�Nm��
Afup	�(��v'}B�`���v�O��I�o�'����<m�@�����u��>��jdh�}2��k�/l����Si���A8��-<Q���+�C[�T�+�#K�[�J�]�aq���3��:�}B�`���v�O+���6�H�G�)�?�'������������/u$�tG���@�t��ce%ww������]x��Mb[�gS������t!VI�����N=s���B���9F>�M���k�_oS�?�Rx���	��)G�*O�!?�%+������&��jV+|�m�[U
N6��>z���t�����v��/�:���
�������
���[s_�_�
>t��I�o�'����<m�@��������,��f��[�������d
V ��r��w�pz����T���v_����3�O���|�1���P�������:��Q����<m�@������'�����������i��u�i�M��hKk&�� �;"7�����9��l����Ps���E�!&��hB�9\�����
��[s_�_�
<��'������������R��J�u����ii�@--�d��W�Q0(H-����|��8�O
��T�E�T�f���P���FK�\�N����Jg���LsG�(v�C�m���(��i�������s��_&�?�n��'���� �?�R��������~'������X��\J���d�De��r	��4�x���>$ML��
.���PK����bc-�2�9��]T���T#��c1�q��^�����V6�� �?�R������?� �?�R��������e�
��L����&vc���W�C����\^Y���Oc)$dub=��U�(��(��(��(��(��(��(��(��(��(��(��(��(��h�_���Y�%�����6�o�Y7�	[��z������7[�m����[=�
����@d��+].]V7��.��V��6���v6��PN5���dN���#4�(��|�`��n������1����c��x�Smv=ZkMh����������������;�(����[jz���Ci�Alo��-s����e��+Q��~���%����5�����2!�0N3�+���8[����K��j��Z��Vwum���Z
��<�<��o�
CI:u���A����p.m�������J�9��k���9�����*��`��35�Y^$���EW.0�)�0�=�>���zm��&�����Kr�$��X�����0<d`��k���9;?=���5���5F������dd+�\��
�u�V'���^j�Z}�����������D�U�a��A��Q@���r��K<RI{rn;x��b%Uv�d�|���&����u�b��e:����g1�5Lg�����8�o��������b+K-xn�����ryb2Q�\�|U��
�����������V���E�3��0!��������-����ovJX��Yu��iv���Rq�Ug�N����(�4�m�l#�f��2:�@���'���@�����j[�v�+��� H>o�;�����b�|=�L����{h��m����Zy��G���p*A�{�k���9i�/�����K�85]?����yC�'�#�����s����o|/o��=���<�=�-�����kv�(��o��Z���������f5�[Q19g`���7a�h��hG[�Hq0��;Q������]v�}���i>*���[
i��J9���[�}���������c�z����G�gx4�J��X�s$n	��J��xUu�^K�n�(��'��c`$�X�VN��mF�X��o�KYI���(m�����.������A�+���N�y���i<���J	o�>/����M�R�{��*����}Ee�>m7����?����c��>�+����V3����F���#J��z=��lm5+yn�t�7q��<K���>MN�]����
��}h7I��i��2�'�N��Dl����w���k+U���������Z���r���
�Jo���_L����?�:~����
*kQq�6��m�@���[g���[��#����������lb)$����Qaq�)~�?�sq��V��y�������_h6�7�t����;_2{�FU����	�FG�
C��&�p�jZ������$���}�HL�o�D�D)�YNCd`��x��M]4�Z( ��R�����6G�����rG&���MKK���������6wq�[c��IJ��� �U='�
��z=�������M�[\E{�2����Qq���Mui�i�imt���R��8�#�@P{�
fP��K�[���5�v�����fm�C|��=sY����a��4�J(�-.����/�J����`� ������9�u��[��[;��+�(�i~irS�s���<Q0�X�b���:�ui���$I'����(�����M�N���������V�o�%�Y[}�7x�?����X]���o��%���=�Z�B�A�kN��Z>�~�W���7M�D������O���cN�R�K��D,�6Oo�U����z}�2O�]]=��yP�X�4������R�z�Z���+_�=�;0P2����
T����w�SE�h������N���pA�U��N�k��^�7�V�yu�0(�;Oq�@z��uB
z�O�"�����������lU![v6����z�����Z�k����kP��N�y#���"���a�����V����{`��R��.�4�/��Q����]J�.�.#��O�$m�?�sq�+X���������e����������GD����m#�����I�*�K����##�V������x@[�]>��&�K���E���T��$�kkP���\������L�+���J�i��i�jW����El7���R�����:xk&�n���3{i��������tC�lX�����I}�=L�K}CR����*�H��i��'�����P�mm��U�$����q�v�}�*_�iq�j6��:��RA)�����0��;�s���SA���L��W�m�[�nm�2]�c�Z����G���oL��g~z�jK�����y�[C����A�@���G�@TV=��-��)�nl�����L&�������x��zf��k0��U�7ims�i�>�v��O���<V�r��r���A��Pp���`��Eq�|<����}f��vY\�a&��7'����������A�nC���t��q��o�����Q@�'���t���qK�.7�|�����Eo���"�n��.��a�������n�@6��MF�S�'��me�C�Ha�����U�1�s���t���5���/
�Z_��:}�\�6}��[����(���m���;=CQ���\��KK#��<��+����R�����P>�llt������m�a�>��7ug�wtP5�x{U�/��L����5��4Z��=���8$�eN3T��w��j:����u��<�a,e��vg#��]��j^�f����uh�-5�>��[����m�T�8�kCL��iz������M>$�����{��[�P<���=?@��K����[9��2� cz����t���G��FY�M��fI��n�ZtPF��G�xF�DvYIj������Ma���m��/���i��E�F�v����u����m|����E���?����-�%o��{�`.�g����/��d���?{�
_�tUv��k���9�!�K=F���Gew������q ��M�����,h�1��?k��)zd���9�n?f��(�����I��E����t�������l`��y�P���.�o�k�Yk�~��ot��#f��*��|b�Z(��i����Y��m�H�c���*�PEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEP\���n�/�[X����[��I21��&�:(��4�B�T���������{f^%��ay���G�\����-��_\Zk����MFmA�b��}�E�.��t�X��<�O������#
i�����
�I}�>~��>�X�|Q>����[�d������!
���Q���0�<����k
']�^��������������mc\~�W���q��5��Ynt9&�o�K�&��YE�q�E"������:��v�P�6���:����v/����1������?!�;`��[N��oX
�P�Z�u��j�0a���F�z5��>�jj�z6�kj'u������3��/�]Y|>�m/�x���H�����\`�AEp^�46�����|��������v�IX���5�gi�G��oBm�_L�������^Z���s�Gn;��Q@G6������)���y�kW3[^�Q�M���9]�dq���WD��O��%��j�R�B��D��0�O�^�Eq:dw^����-������/ �]JE���f�����x�J��)���Z��_�����Y���������W��@d6�v~.�����z�Q5��q����9�NG��X�>���~�F���K�����E
+�m*��
�8`@$w���(�<-�mCO������[X��u����.����V����aq��jQ\@�+�WD�`f�#��CEy��������D���e�2�[j6ryQ���dv���a�qZvq��&�n5=�T:������_c
��$m�#��]��������(������P�4��td��
?��������#��o�]jI��{v���m�����C�G����w4P�E�k�h^����V�"�-5�$�F<�����Z������
������{�	�o$�2��I��<�_���uw�Om`���������Q*oE��C�Ek�v����Z����>�c5��
��y�Fv2����q]�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE��
#33Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#32)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Tue, Feb 25, 2025 at 3:26 PM Ajin Cherian <itsajin@gmail.com> wrote:

On Fri, Feb 21, 2025 at 2:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

On Fri, Feb 21, 2025 at 7:57 AM Ajin Cherian <itsajin@gmail.com> wrote:

In these tests, I also see an increased performance with the patch
even when all transactions are published. I will investigate why this
happens and update.

Yes, it is important to investigate this because in the best case, it
should match with HEAD. One thing you can verify is whether the
changes processed on the server are exactly for the published table,
it shouldn't happen that it is processing both published and
unpublished changes. If the server is processing for both tables then
it is expected that the patch performs better. I think you can verify
before starting each test and after finishing each test whether the
slot is pointing at the appropriate location for the next test or
create a new slot for each with the required location.

Yes, you are right, I modified the tests to drop the slot and create a
new slot advance to current_lsn and now I see a fractionally better
performance in head code when all transactions are published.
Graph attached.

Just to summarize and clarify:

The patched code significantly improves performance when no
transactions are published, reducing execution time from ~54,100 ms
(head code) to ~17,700 ms—a nearly 70% improvement.
When half of the transactions are published, the patched code also
shows a notable performance gain, reducing execution time from ~62,800
ms to ~44,500 ms (~29% faster).
When all transactions are published, the patched code is only 0.53%
slower than the head code, indicating a negligible performance
degradation.

regards,
Ajin Cherian
Fujitsu Australia

#34Ajin Cherian
itsajin@gmail.com
In reply to: Hayato Kuroda (Fujitsu) (#24)
3 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Thu, Feb 20, 2025 at 8:42 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:

Dear Ajin,

Here are my comments. I must play with patches to understand more.

01.
```
extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
XLogRecPtr lsn, RelFileLocator *rlocator,
ReorderBufferChangeType change_type,
bool has_tuple);
```

Can you explain why "has_tuple is needed? All callers is set to true.

It is not needed. Removed.

02.
```
static Relation
ReorderBufferGetRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
bool has_tuple)
```

Hmm, the naming is bit confusing for me. This operation is mostly not related with
the reorder buffer. How about "GetPossibleDecodableRelation" or something?

03.
```
if (IsToastRelation(relation))
{
Oid real_reloid = InvalidOid;
char *toast_name = RelationGetRelationName(relation);
/* pg_toast_ len is 9 */
char *start_ch = &toast_name[9];

real_reloid = pg_strtoint32(start_ch);
entry->relid = real_reloid;
}
```

It is bit hacky for me. How about using sscanf like attached?

I have used the code that you shared to modify this.

04.

IIUC, toast tables always require the filter_change() call twice, is it right?
I understood like below:

1. ReorderBufferFilterByRelFileLocator() tries to filter the change at outside the
transaction. The OID indicates the pg_toast_xxx table.
2. pgoutput_filter_change() cannot find the table from the hash. It returns false
with cache_valid=false.
3. ReorderBufferFilterByRelFileLocator() starts a transaction and get its relation.
4. The function recognizes the relation seems toast and get parent oid.
5. The function tries to filter the change in the transaction, with the parent oid.
6. pgoutput_filter_change()->get_rel_sync_entry() enters the parent relation to the
hash and return determine the filtable or not.
7. After sometime, the same table is modified. But the toast table is not stored in
the hash so that whole 1-6 steps are required.

I feel this may affect the perfomance when many toast is modified. How about skiping
the check for toasted ones? ISTM IsToastNamespace() can be used for the decision.

I understand your concern but I also think that the benefits are
higher for toast changes

On Fri, Feb 21, 2025 at 6:26 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v14-0001.

======
src/backend/replication/logical/reorderbuffer.c

1.
+ *   We also try and filter changes that are not relevant for logical decoding
+ *   as well as give the option for plugins to filter changes in advance.
+ *   Determining whether to filter a change requires information about the
+ *   relation from the catalog, requring a transaction to be started.
+ *   When most changes in a transaction are unfilterable, the overhead of
+ *    starting a transaction for each record is significant. To reduce this
+ *    overhead a hash cache of relation file locators is created. Even then a
+ *    hash search for every record before recording has considerable overhead
+ *    especially for use cases where most tables in an instance are
not filtered.
+ *    To further reduce this overhead a simple approach is used to suspend
+ *    filtering for a certain number of changes CHANGES_THRESHOLD_FOR_FILTER
+ *    when an unfilterable change is encountered. In other words, continue
+ *    filtering changes if the last record was filtered out. If an unfilterable
+ *    change is found, skip filtering the next CHANGES_THRESHOLD_FOR_FILTER
+ *    changes.
+ *

1a.
/try and filter/try to filter/

~

1b.
There is some leading whitespace problem happening (spaces instead of tabs?)

~

1c.
Minor rewording

SUGGESTION (e.g. anyway this should be identical to the commit message text)

Determining whether to filter a change requires information about the
relation and the publication from the catalog, which means a
transaction must be started. But, the overhead of starting a
transaction for each record is significant. To reduce this overhead a
hash cache of relation file locators is used to remember which
relations are filterable.

Even so, doing a hash search for every record has considerable
overhead, especially for scenarios where most tables in an instance
are published. To further reduce overheads a simple approach is used:
When an unfilterable change is encountered we suspend filtering for a
certain number (CHANGES_THRESHOLD_FOR_FILTER) of subsequent changes.
In other words, continue filtering until an unfilterable change is
encountered; then skip filtering the next CHANGES_THRESHOLD_FOR_FILTER
changes, before attempting filtering again.

~~~

I have modified the whole commit message and these comments based on
inputs from Hou-san.

2.
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100

Maybe you can explain where this magic number comes from.

SUGGESTION
The CHANGES_THRESHOLD_FOR_FILTER value of 100 was chosen as the best
trade-off value after performance tests were carried out using
candidate values 10, 50, 100, and 200.

I am not so sure as my test scenarios where very limited, I don't
think that I can say that 100 is the best value for all scenarios.
Which is why i suggest changing this as a user changable GUC variable
if others agree.

~~~

ReorderBufferQueueChange:

3.
+ /*
+ * If filtering was suspended and we've crossed the change threshold,
+ * attempt to filter again
+ */
+ if (!rb->can_filter_change && (++rb->unfiltered_changes_count
+ >= CHANGES_THRESHOLD_FOR_FILTER))
+ {
+ rb->can_filter_change = true;
+ rb->unfiltered_changes_count = 0;
+ }
+

/If filtering was suspended/If filtering is currently suspended/

~~~

Fixed.

ReorderBufferGetRelation:

4.
+static Relation
+ReorderBufferGetRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+ bool has_tuple)

My suggested [2-#4] name change 'ReorderBufferGetRelationForDecoding'
is not done yet. I saw Kuroda-san also said this name was confusing
[1-#02], and suggested something similar
'GetPossibleDecodableRelation'.

Changed it to GetDecodableRelation()

~~~

RelFileLocatorCacheInvalidateCallback:

5.
+ /*
+ * If relid is InvalidOid, signaling a complete reset, we must remove
+ * all entries, otherwise just remove the specific relation's entry.
+ * Always remove negative cache entries.
+ */
+ if (relid == InvalidOid || /* complete reset */
+ entry->relid == InvalidOid || /* invalid cache entry */
+ entry->relid == relid) /* individual flushed relation */
+ {
+ if (hash_search(RelFileLocatorFilterCache,
+ &entry->key,
+ HASH_REMOVE,
+ NULL) == NULL)
+ elog(ERROR, "hash table corrupted");
+ }

5a.
IMO the relid *parameter* should be mentioned explicitly to
disambiguate relid from the entry->relid.

/If relid is InvalidOid, signaling a complete reset,/If a complete
reset is requested (when 'relid' parameter is InvalidOid),/

~

5b.
/Always remove negative cache entries./Remove any invalid cache
entries (these are indicated by invalid entry->relid)/

~~~

Fixed

ReorderBufferFilterByRelFileLocator:

6.
I previously [2-#7] had suggested this function code could be
refactored to share some common return logic. It is not done, but OTOH
there is no reply, so I don't know if it was overlooked or simply
rejected.

I had considered this but I felt that it is better readable this way.

======
src/include/replication/reorderbuffer.h

7.
+ /* should we try to filter the change? */
+ bool can_filter_change;
+
+ /* number of changes after a failed attempt at filtering */
+ int8 unfiltered_changes_count;
+

The potential renaming of that 'can_filter_change' field to something
better is still an open item IMO [2-#8] pending consensus on what a
better name for this might be.

I haven't changed this.

On Sun, Feb 23, 2025 at 2:19 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for v14-0002.

======
src/backend/replication/logical/decode.c

1.
There is lots of nearly duplicate code checking to see if a change is filterable

DecodeInsert:
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
- buf->origptr, &target_locator, true))
+ buf->origptr, &target_locator,
+ REORDER_BUFFER_CHANGE_INSERT,
+ true))
DecodeUpdate:
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+ buf->origptr, &target_locator,
+ REORDER_BUFFER_CHANGE_UPDATE,
+ true))
+ return;
+
DecodeDelete:
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+ buf->origptr, &target_locator,
+ REORDER_BUFFER_CHANGE_DELETE,
+ true))
+ return;
+
DecodeMultiInsert:
/*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+ buf->origptr, &rlocator,
+ REORDER_BUFFER_CHANGE_INSERT,
+ true))
+ return;
+
DecodeSpecConfirm:
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+ buf->origptr, &target_locator,
+ REORDER_BUFFER_CHANGE_INSERT,
+ true))
+ return;
+

Can't all those code fragments (DecodeInsert, DecodeUpdate,
DecodeDelete, DecodeMultiInsert, DecodeSpecConfirm) delegate to a
new/common 'SkipThisChange(...)' function?

Fixed this accordingly.

======
src/backend/replication/logical/reorderbuffer.c

2.
/*
* After encountering a change that cannot be filtered out, filtering is
- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing
CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
* This strategy helps to minimize the overhead of performing a hash table
* search for each record, especially when most changes are not filterable.
*/
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0

Why is this defined as 0? Some accidental residue from performance
testing different values?

I need it to remain 0 for my tests to work. I have mooved out these
changes and tests to a new patch 0003.

======
src/test/subscription/t/001_rep_changes.pl

3.
+# Check that an unpublished change is filtered out.
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+ 'unpublished INSERT is filtered');
+
+ok($logfile =~ qr/Filtering UPDATE/,
+ 'unpublished UPDATE is filtered');
+
+ok($logfile =~ qr/Filtering DELETE/,
+ 'unpublished DELETE is filtered');
+

AFAICT these are probably getting filtered out because the entire
table is not published at all.

Should you also add different tests where you do operations on a table
that IS partially replicated but only some of the *operations* are not
published. e.g. test the different 'pubactions' of the PUBLICATION
'publish' parameter. Maybe you need different log checks to
distinguish the different causes for the filtering.

I've modified the commit message in 0001 based on inputs from Hou-san.
Modified and added new tests in patch 0003

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

v15-0003-Tests-for-filtering-unpublished-changes.patchapplication/octet-stream; name=v15-0003-Tests-for-filtering-unpublished-changes.patchDownload
From 188c4f25406bfa02f93594e1a660f82f01939f1c Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Mon, 3 Mar 2025 04:56:50 -0500
Subject: [PATCH v15 3/3] Tests for filtering unpublished changes

---
 src/backend/replication/logical/reorderbuffer.c |  2 +-
 src/test/subscription/t/001_rep_changes.pl      | 65 +++++++++++++++++++++++++
 2 files changed, 66 insertions(+), 1 deletion(-)
 mode change 100644 => 100755 src/test/subscription/t/001_rep_changes.pl

diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 7a7d775..3884900 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -376,7 +376,7 @@ static void ReorderBufferMemoryResetcallback(void *arg);
  * This strategy helps to minimize the overhead of performing a hash table
  * search for each record, especially when most changes are not filterable.
  */
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0
 
 /*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
diff --git a/src/test/subscription/t/001_rep_changes.pl b/src/test/subscription/t/001_rep_changes.pl
old mode 100644
new mode 100755
index 8726fe0..c49aff4
--- a/src/test/subscription/t/001_rep_changes.pl
+++ b/src/test/subscription/t/001_rep_changes.pl
@@ -555,6 +555,71 @@ $result = $node_subscriber->safe_psql('postgres',
 	"SELECT count(*) FROM pg_replication_origin");
 is($result, qq(0), 'check replication origin was dropped on subscriber');
 
+$node_publisher->stop('fast');
+
+# Additional tests for filtering of unpublished changes
+# Bump up log verbosity to DEBUG1 for confirmation logs
+$node_publisher->append_conf('postgresql.conf', "log_min_messages = debug1");
+$node_publisher->start;
+
+# Create new tables on publisher and subscriber
+$node_publisher->safe_psql('postgres', "CREATE TABLE pub_table (id int primary key, data text)");
+$node_publisher->safe_psql('postgres', "CREATE TABLE unpub_table (id int primary key, data text)");
+$node_publisher->safe_psql('postgres', "CREATE TABLE insert_only_table (id int primary key, data text)");
+$node_publisher->safe_psql('postgres', "CREATE TABLE delete_only_table (id int primary key, data text)");
+
+$node_subscriber->safe_psql('postgres', "CREATE TABLE pub_table (id int primary key, data text)");
+$node_subscriber->safe_psql('postgres', "CREATE TABLE unpub_table (id int primary key, data text)");
+$node_subscriber->safe_psql('postgres', "CREATE TABLE insert_only_table (id int primary key, data text)");
+$node_subscriber->safe_psql('postgres', "CREATE TABLE delete_only_table (id int primary key, data text)");
+
+
+# Setup logical replication publications
+$node_publisher->safe_psql('postgres', "CREATE PUBLICATION pub_all FOR TABLE pub_table");
+$node_publisher->safe_psql('postgres', "CREATE PUBLICATION pub_insert_only FOR TABLE insert_only_table WITH (publish = insert)");
+$node_publisher->safe_psql('postgres', "CREATE PUBLICATION pub_delete_only FOR TABLE delete_only_table WITH (publish = delete)");
+
+# Setup logical replication subscription
+$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION sub_all CONNECTION '$publisher_connstr' PUBLICATION pub_all");
+$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION sub_insert_only CONNECTION '$publisher_connstr' PUBLICATION pub_insert_only");
+$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION sub_delete_only CONNECTION '$publisher_connstr' PUBLICATION pub_delete_only");
+
+# Wait for initial sync
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_all');
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_insert_only');
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_delete_only');
+
+# Insert into an unpublished table (should not be replicated)
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO unpub_table VALUES (1, 'unpublished')");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+
+# Insert, delete, and update tests for restricted publication tables
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO insert_only_table VALUES (1, 'to be inserted')");
+$node_publisher->safe_psql('postgres', "UPDATE insert_only_table SET data = 'updated' WHERE id = 1");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering UPDATE/,
+	'unpublished UPDATE is filtered');
+
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "DELETE FROM insert_only_table WHERE id = 1");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering DELETE/,
+	'unpublished DELETE is filtered');
+
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO delete_only_table VALUES (1, 'to be deleted')");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_all");
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_insert_only");
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_delete_only");
+
 $node_subscriber->stop('fast');
 $node_publisher->stop('fast');
 
-- 
1.8.3.1

v15-0002-Introduce-a-output-plugin-callback-to-filter-cha.patchapplication/octet-stream; name=v15-0002-Introduce-a-output-plugin-callback-to-filter-cha.patchDownload
From b4e23aec71a2a2193f5d94fa9b9c2285e1f1d25a Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Mon, 3 Mar 2025 04:55:13 -0500
Subject: [PATCH v15 2/3] Introduce a output plugin callback to filter changes

This new output plugin callback provides an option to logical decoding plugins to filter out
changes early. The primary purpose of the callback is to conserve memory and processing cycles
by excluding changes that are not required by output plugins.
---
 doc/src/sgml/logicaldecoding.sgml               | 41 ++++++++++++++-
 src/backend/replication/logical/decode.c        | 66 +++++++++++++++++++++--
 src/backend/replication/logical/logical.c       | 42 +++++++++++++++
 src/backend/replication/logical/reorderbuffer.c | 49 +++++++++++++----
 src/backend/replication/pgoutput/pgoutput.c     | 70 +++++++++++++++++++++++++
 src/include/replication/output_plugin.h         | 20 +++++++
 src/include/replication/reorderbuffer.h         | 10 +++-
 src/test/subscription/t/013_partition.pl        |  7 +++
 8 files changed, 289 insertions(+), 16 deletions(-)

diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 1c4ae38..a603c47 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -560,6 +560,7 @@ typedef struct OutputPluginCallbacks
     LogicalDecodeCommitCB commit_cb;
     LogicalDecodeMessageCB message_cb;
     LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+    LogicalDecodeFilterChangeCB filter_change_cb;
     LogicalDecodeShutdownCB shutdown_cb;
     LogicalDecodeFilterPrepareCB filter_prepare_cb;
     LogicalDecodeBeginPrepareCB begin_prepare_cb;
@@ -582,8 +583,8 @@ typedef void (*LogicalOutputPluginInit) (struct OutputPluginCallbacks *cb);
      and <function>commit_cb</function> callbacks are required,
      while <function>startup_cb</function>, <function>truncate_cb</function>,
      <function>message_cb</function>, <function>filter_by_origin_cb</function>,
-     and <function>shutdown_cb</function> are optional.
-     If <function>truncate_cb</function> is not set but a
+     <function>shutdown_cb</function>, and <function>filter_change_cb</function>
+     are optional. If <function>truncate_cb</function> is not set but a
      <command>TRUNCATE</command> is to be decoded, the action will be ignored.
     </para>
 
@@ -871,6 +872,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
      </para>
      </sect3>
 
+     <sect3 id="logicaldecoding-output-plugin-filter-change">
+     <title>Change Filter Callback</title>
+
+     <para>
+      The optional <function>filter_change_cb</function> is called before a
+      change record is decoded to determine whether the change can be filtered
+      out.
+<programlisting>
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+                                             Oid relid,
+                                             ReorderBufferChangeType change_type,
+                                             bool in_txn, bool *cache_valid);
+</programlisting>
+      To indicate that decoding can be skipped for the given change
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.
+      The <parameter>in_txn</parameter> parameter indicates whether the
+      callback is invoked within a transaction block.
+      When <parameter>in_txn</parameter> is false, and if making a decision to filter a change requires being inside a
+      transaction block, such as needing access to the catalog, set
+      <parameter>*cache_valid</parameter> to <literal>false</literal>.
+      This ensures that the callback will be reinvoked once a transaction block
+      starts. If a decision can be made immediately, set
+      <parameter>*cache_valid</parameter> to <literal>true</literal>.
+     </para>
+     <para>
+      The primary purpose of this callback function is to optimize memory usage
+      and processing efficiency by filtering out changes that are unnecessary for
+      output plugins. It enables output plugins to selectively process relevant
+      changes. Caching filtering decisions locally is recommended, as it enables
+      the callback to provide cached results without repeatedly initiating
+      transactions or querying the catalog. This approach minimizes overhead
+      and improves efficiency during the decoding process.
+     </para>
+     </sect3>
+
     <sect3 id="logicaldecoding-output-plugin-message">
      <title>Generic Message Callback</title>
 
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index d8e0114..fc6219f 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -576,6 +576,17 @@ FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
 }
 
 /*
+ * Check if filtering changes before decoding is supported and we're not suppressing filter
+ * changes currently.
+ */
+static inline bool
+FilterChangeIsEnabled(LogicalDecodingContext *ctx)
+{
+	return (ctx->callbacks.filter_change_cb != NULL &&
+				ctx->reorder->can_filter_change);
+}
+
+/*
  * Handle rmgr LOGICALMSG_ID records for LogicalDecodingProcessRecord().
  */
 void
@@ -881,6 +892,15 @@ DecodeAbort(LogicalDecodingContext *ctx, XLogRecordBuffer *buf,
 	UpdateDecodingStats(ctx);
 }
 
+/* Function to determine whether to skip the change */
+static bool SkipThisChange(LogicalDecodingContext *ctx, XLogRecPtr origptr, TransactionId xid, 
+                           RelFileLocator *target_locator, ReorderBufferChangeType change_type)
+{
+	return (FilterChangeIsEnabled(ctx) &&
+			ReorderBufferFilterByRelFileLocator(ctx->reorder, xid, origptr, target_locator,
+												change_type));
+}
+
 /*
  * Parse XLOG_HEAP_INSERT (not MULTI_INSERT!) records into tuplebufs.
  *
@@ -915,9 +935,13 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
-	if (ctx->reorder->can_filter_change &&
-		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
-											buf->origptr, &target_locator))
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+						REORDER_BUFFER_CHANGE_INSERT))	
 		return;
 
 	change = ReorderBufferGetChange(ctx->reorder);
@@ -970,6 +994,15 @@ DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator, 
+						REORDER_BUFFER_CHANGE_UPDATE))
+        return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_UPDATE;
 	change->origin_id = XLogRecGetOrigin(r);
@@ -1036,6 +1069,15 @@ DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+    if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator, 
+                        REORDER_BUFFER_CHANGE_DELETE))
+        return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 
 	if (xlrec->flags & XLH_DELETE_IS_SUPER)
@@ -1139,6 +1181,15 @@ DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 		return;
 
 	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+    if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &rlocator, 
+                        REORDER_BUFFER_CHANGE_INSERT))
+        return;
+
+	/*
 	 * We know that this multi_insert isn't for a catalog, so the block should
 	 * always have data even if a full-page write of it is taken.
 	 */
@@ -1229,6 +1280,15 @@ DecodeSpecConfirm(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+    if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator, 
+                        REORDER_BUFFER_CHANGE_INSERT))
+        return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM;
 	change->origin_id = XLogRecGetOrigin(r);
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 8ea846b..635e1de 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -74,6 +74,9 @@ static void truncate_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 							   XLogRecPtr message_lsn, bool transactional,
 							   const char *prefix, Size message_size, const char *message);
+static bool filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+								  ReorderBufferChangeType change_type, bool in_txn,
+								  bool *cache_valid);
 
 /* streaming callbacks */
 static void stream_start_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
@@ -220,6 +223,7 @@ StartupDecodingContext(List *output_plugin_options,
 	/* wrap output plugin callbacks, so we can add error context information */
 	ctx->reorder->begin = begin_cb_wrapper;
 	ctx->reorder->apply_change = change_cb_wrapper;
+	ctx->reorder->filter_change = filter_change_cb_wrapper;
 	ctx->reorder->apply_truncate = truncate_cb_wrapper;
 	ctx->reorder->commit = commit_cb_wrapper;
 	ctx->reorder->message = message_cb_wrapper;
@@ -1224,6 +1228,44 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+static bool
+filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+					  ReorderBufferChangeType change_type, bool in_txn,
+					  bool *cache_valid)
+{
+	LogicalDecodingContext *ctx = cache->private_data;
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	/* check if the filter change callback is supported */
+	if (ctx->callbacks.filter_change_cb == NULL)
+		return false;
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_change";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_change_cb(ctx, relid, change_type, in_txn, cache_valid);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 5c30cf0..7a7d775 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -371,7 +371,8 @@ static void ReorderBufferMemoryResetcallback(void *arg);
 
 /*
  * After encountering a change that cannot be filtered out, filtering is
- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
  * This strategy helps to minimize the overhead of performing a hash table
  * search for each record, especially when most changes are not filterable.
  */
@@ -5703,8 +5704,8 @@ ReorderBufferMemoryResetcallback(void *arg)
  */
 bool
 ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
-									XLogRecPtr lsn, RelFileLocator *rlocator)
-
+									XLogRecPtr lsn, RelFileLocator *rlocator,
+									ReorderBufferChangeType change_type)
 {
 	bool		found;
 	Relation	relation;
@@ -5720,7 +5721,6 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
 	Assert(txn);
 	toptxn = rbtxn_get_toptxn(txn);
-	rb->can_filter_change = false;
 
 	/*
 	 * We cannot construct an accurate historical snapshot until all pending
@@ -5737,7 +5737,25 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (found)
 	{
 		rb->can_filter_change = entry->filterable;
-		return entry->filterable;
+
+		/*
+		 * Quick return if we already know that the relation is not to be decoded.
+		 * These are for special relations that are unlogged and for sequences
+		 * and catalogs.
+		 */
+		if (entry->filterable)
+			return true;
+
+		/* Allow the output plugin to filter relations */
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  false, &cache_valid);
+
+		/*
+		 * If plugin had the relation ready in cache, the response is valid, else
+		 * we'll need to call the plugin a second time within a transaction.
+		 */
+		if (cache_valid)
+			return rb->can_filter_change;
 	}
 
 	/* constructs a temporary historical snapshot */
@@ -5761,18 +5779,29 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 
 	if (RelationIsValid(relation))
 	{
-		entry->relid = RelationGetRelid(relation);
+		if (IsToastRelation(relation))
+		{
+			char   *toast_name = RelationGetRelationName(relation);
+			int		n PG_USED_FOR_ASSERTS_ONLY;
+
+			n = sscanf(toast_name, "pg_toast_%u", &entry->relid);
+
+			Assert(n == 1);
+		}
+		else
+			entry->relid = RelationGetRelid(relation);
+
 		entry->filterable = false;
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  true, &cache_valid);
 		RelationClose(relation);
 	}
 	else
 	{
 		entry->relid = InvalidOid;
-		entry->filterable = true;
+		rb->can_filter_change = entry->filterable = true;
 	}
 
-	rb->can_filter_change = entry->filterable;
-
 	ReorderBufferFreeSnap(rb, snapshot_now);
 
 	TeardownHistoricSnapshot(false);
@@ -5788,5 +5817,5 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (using_subtxn)
 		RollbackAndReleaseCurrentSubTransaction();
 
-	return entry->filterable;
+	return rb->can_filter_change;
 }
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index 7d464f6..93eb579 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -57,6 +57,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+								   ReorderBufferChangeType change_type, bool in_txn,
+								   bool *cache_valid);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -268,6 +271,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_change_cb = pgoutput_filter_change;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1744,6 +1748,72 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 }
 
 /*
+ * Determine whether a change to the specified relation should be published.
+ *
+ * See the comments atop of LogicalDecodeFilterChangeCB for details.
+ */
+static bool
+pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+					   ReorderBufferChangeType change_type, bool in_txn, bool *cache_valid)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *entry;
+
+	Assert(RelationSyncCache != NULL);
+
+	if (in_txn)
+	{
+		Relation	relation;
+
+		relation = RelationIdGetRelation(relid);
+		entry = get_rel_sync_entry(data, relation);
+		*cache_valid = true;
+	}
+	else
+	{
+		entry = (RelationSyncEntry *) hash_search(RelationSyncCache,
+												  &relid,
+												  HASH_FIND, cache_valid);
+		if (!*cache_valid)
+			return false;
+	}
+
+	/*
+	 * If the pubaction is not supported by this publication then return true to say
+	 * the change for this entry can be skipped.
+	 */
+	switch (change_type)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			if (!entry->pubactions.pubinsert)
+			{
+				elog(DEBUG1, "Filtering INSERT");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			if (!entry->pubactions.pubupdate)
+			{
+				elog(DEBUG1, "Filtering UPDATE");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			if (!entry->pubactions.pubdelete)
+			{
+				elog(DEBUG1, "Filtering DELETE");
+				return true;
+			}
+			break;
+		default:
+			/* allow any other changes that are not explicitly filtered */
+			return false;
+	}
+
+	return false;
+}
+
+/*
  * Shutdown the output plugin.
  *
  * Note, we don't need to clean the data->context, data->cachectx, and
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 8d4d5b71..f82bc1d 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -97,6 +97,25 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
 											   RepOriginId origin_id);
 
 /*
+ * This callback is called before a change record is decoded to determine
+ * whether the change can be filtered out before entering the reorder buffer.
+ *
+ * Typically, the output plugin needs to refer to the system catalog to make
+ * this decision. To enhance efficiency, the plugin should cache the lookup
+ * result for each relation, minimizing catalog access on subsequent calls.
+ *
+ * If the callback is called with 'in_txn' set as false (the reorder
+ * buffer has not started a transaction), the output plugin should set '*cache_valid'
+ * to false to indicate that the result is not available in its internal cache.
+ * If 'in_txn' is true, the plugin can create a cache entry after querying the
+ * catalog.
+ */
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
+/*
  * Called to shutdown an output plugin.
  */
 typedef void (*LogicalDecodeShutdownCB) (struct LogicalDecodingContext *ctx);
@@ -222,6 +241,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterChangeCB filter_change_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 24166c2..d45b261 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -494,6 +494,12 @@ typedef void (*ReorderBufferMessageCB) (ReorderBuffer *rb,
 										const char *prefix, Size sz,
 										const char *message);
 
+/* filter change callback signature */
+typedef bool (*ReorderBufferFilterChangeCB) (ReorderBuffer *rb,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
 /* begin prepare callback signature */
 typedef void (*ReorderBufferBeginPrepareCB) (ReorderBuffer *rb,
 											 ReorderBufferTXN *txn);
@@ -604,6 +610,7 @@ struct ReorderBuffer
 	 */
 	ReorderBufferBeginCB begin;
 	ReorderBufferApplyChangeCB apply_change;
+	ReorderBufferFilterChangeCB filter_change;
 	ReorderBufferApplyTruncateCB apply_truncate;
 	ReorderBufferCommitCB commit;
 	ReorderBufferMessageCB message;
@@ -776,7 +783,8 @@ extern void ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr);
 extern void StartupReorderBuffer(void);
 
 extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
-												XLogRecPtr lsn, RelFileLocator *rlocator);
+												XLogRecPtr lsn, RelFileLocator *rlocator,
+												ReorderBufferChangeType change_type);
 extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
 
 #endif
diff --git a/src/test/subscription/t/013_partition.pl b/src/test/subscription/t/013_partition.pl
index 61b0cb4..a64b27e 100644
--- a/src/test/subscription/t/013_partition.pl
+++ b/src/test/subscription/t/013_partition.pl
@@ -472,6 +472,13 @@ $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4 (a int PRIMARY KEY)");
 $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4_1 (a int PRIMARY KEY)");
+
+# Ensure that the subscription 'sub2' catches up with the latest changes. This
+# is necessary for the walsender to update its historic snapshot. Otherwise,
+# the walsender might retain an outdated snapshot, potentially preventing it
+# from accessing the newly created publication.
+$node_publisher->wait_for_catchup('sub2');
+
 # Since we specified publish_via_partition_root in pub_all and
 # pub_lower_level, all partition tables use their root tables' identity and
 # schema. We set the list of publications so that the FOR ALL TABLES
-- 
1.8.3.1

v15-0001-Filter-transactions-that-need-not-be-published.patchapplication/octet-stream; name=v15-0001-Filter-transactions-that-need-not-be-published.patchDownload
From 27d7c257dc37efdd31a915efe2ff7220ac323409 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Mon, 3 Mar 2025 04:52:05 -0500
Subject: [PATCH v15 1/3] Filter transactions that need not be published.

This patch set aims to filter transactions at the decode stage rather than
streaming time, which allows the system to skip processing transactions that do
not contain tables included in the logical replication walsender's publication
list. As a result, this can prevent the serialization of unnecessary records to
disk during non-streaming mode, especially in large transactions, and also
reduce memory and CPU usage in streaming mode when many changes can be filtered
out.

A hash cache of relation file locators is implemented to cache whether a
relation is filterable or not. This ensures that we only need to retrieve
relation and publication information from the catalog when a cache entry is
invalid, avoiding the overhead of starting a transaction for each record.

Filtering is temporarily suspended for a sequence of changes (set to 100
changes) when an unfilterable change is encountered. This strategy minimizes
the overhead of hash searching for every record, which is beneficial when the
majority of tables in an instance are published and thus unfilterable. The
threshold of 100 was determined to be the optimal balance based on performance
tests.

Additionally, filtering is paused for transactions containing WAL records
(INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION) that modify the historical
snapshot constructed during logical decoding. This pause is necessary because
constructing a correct historical snapshot for searching publication
information requires processing these WAL records.

Note that this patch filters changes only for system catalog relations,
non-logically logged relations, or temporary heaps and sequences. A subsequent
patch will introduce a new output plugin in pgoutput, which will further filter
changes for relations not included in publications.
---
 src/backend/replication/logical/decode.c        |   5 +
 src/backend/replication/logical/reorderbuffer.c | 467 ++++++++++++++++++++----
 src/include/replication/reorderbuffer.h         |  18 +
 src/tools/pgindent/typedefs.list                |   2 +
 4 files changed, 423 insertions(+), 69 deletions(-)

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 24d88f3..d8e0114 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	if (ctx->reorder->can_filter_change &&
+		ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+											buf->origptr, &target_locator))
+		return;
+
 	change = ReorderBufferGetChange(ctx->reorder);
 	if (!(xlrec->flags & XLH_INSERT_IS_SPECULATIVE))
 		change->action = REORDER_BUFFER_CHANGE_INSERT;
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 5186ad2..5c30cf0 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -82,6 +82,32 @@
  *	  a bit more memory to the oldest subtransactions, because it's likely
  *	  they are the source for the next sequence of changes.
  *
+ *	  We also try to filter changes at the decode stage rather than at
+ *	  streaming time, which allows the system to skip processing txns that
+ *	  do not contain tables that need to be decoded. As a result,
+ *	  this can prevent the serialization of unnecessary records to disk
+ *	  during non-streaming mode, especially in large transactions, and also
+ *	  reduce memory and CPU usage in streaming mode when many changes can be
+ *	  filtered out.
+ *
+ *	  A hash cache of relation file locators is implemented to cache whether a
+ *	  relation is filterable or not. This ensures that we only need to retrieve
+ *	  relation and publication information from the catalog when a cache entry is
+ *	  invalid, avoiding the overhead of starting a transaction for each record.
+ *
+ *	  Filtering is temporarily suspended for a sequence of changes (set to 100
+ *	  changes) when an unfilterable change is encountered. This strategy minimizes
+ *	  the overhead of hash searching for every record, which is beneficial when the
+ *	  majority of tables in an instance are published and thus unfilterable. The
+ *	  threshold of 100 was determined to be the optimal balance based on performance
+ *	  tests.
+ *
+ *	  Additionally, filtering is paused for transactions containing WAL records
+ *	  (INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION) that modify the historical
+ *	  snapshot constructed during logical decoding. This pause is necessary because
+ *	  constructing a correct historical snapshot for searching publication
+ *	  information requires processing these WAL records.
+ *
  * -------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -109,6 +135,7 @@
 #include "storage/procarray.h"
 #include "storage/sinval.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 #include "utils/relfilenumbermap.h"
@@ -227,8 +254,10 @@ static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
 											   XLogRecPtr lsn, bool create_as_top);
 static void ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
 											  ReorderBufferTXN *subtxn);
-
 static void AssertTXNLsnOrder(ReorderBuffer *rb);
+static Relation GetDecodableRelation(ReorderBuffer *rb,
+										 RelFileLocator *rlocator,
+										 bool has_tuple);
 
 /* ---------------------------------------
  * support functions for lsn-order iterating over the ->changes of a
@@ -279,6 +308,8 @@ static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
  */
 static inline bool ReorderBufferCanStream(ReorderBuffer *rb);
 static inline bool ReorderBufferCanStartStreaming(ReorderBuffer *rb);
+static Snapshot ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb,
+											   ReorderBufferTXN *txn);
 static void ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
 static void ReorderBufferStreamCommit(ReorderBuffer *rb, ReorderBufferTXN *txn);
 
@@ -305,6 +336,48 @@ static void ReorderBufferChangeMemoryUpdate(ReorderBuffer *rb,
 											bool addition, Size sz);
 
 /*
+ * ---------------------------------------
+ * RelFileLocator filtering
+ * ---------------------------------------
+ * This hash table serves as a lookup table for determining if a relation can
+ * be filtered before being decoded and queued into the buffer.
+ *
+ * The hash table shares the same lifespan as the reorder buffer. This is
+ * crucial because each reorderbuffer may have different configurations or be
+ * associated with different output plugins, affecting the types of tables to
+ * be processed.
+ */
+
+static HTAB *RelFileLocatorFilterCache = NULL;
+
+static bool relation_callbacks_registered = false;
+
+typedef struct ReorderBufferRelFileLocatorKey
+{
+	Oid			reltablespace;
+	RelFileNumber relfilenumber;
+} ReorderBufferRelFileLocatorKey;
+
+/* Hash table entry used to determine if the relation can be filtered. */
+typedef struct ReorderBufferRelFileLocatorEnt
+{
+	ReorderBufferRelFileLocatorKey key;
+	Oid			relid;
+	bool		filterable;
+} ReorderBufferRelFileLocatorEnt;
+
+static void RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid);
+static void ReorderBufferMemoryResetcallback(void *arg);
+
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100
+
+/*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
  * prior ReorderBuffer instances for the same slot.
  */
@@ -362,6 +435,34 @@ ReorderBufferAllocate(void)
 	buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
 								 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
+	/*
+	 * To support early filtering of changes, this hash table acts
+	 * as a lookup table to determine if the corresponding relation is
+	 * required to be decoded and queued into the buffer. The hash table
+	 * shares the same lifespan as the reorder buffer.
+	 *
+	 * Also setup the callback to invalidate cache when relations are updated.
+	 */
+	hash_ctl.keysize = sizeof(ReorderBufferRelFileLocatorKey);
+	hash_ctl.entrysize = sizeof(ReorderBufferRelFileLocatorEnt);
+	hash_ctl.hcxt = buffer->context;
+
+	RelFileLocatorFilterCache =
+		hash_create("RelFileLocatorFilterCache", 1000, &hash_ctl,
+					HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	buffer->relfile_callback.arg = NULL;
+	buffer->relfile_callback.func = ReorderBufferMemoryResetcallback;
+	MemoryContextRegisterResetCallback(buffer->context, &buffer->relfile_callback);
+
+	if (!relation_callbacks_registered)
+	{
+		/* Watch for invalidation events. */
+		CacheRegisterRelcacheCallback(RelFileLocatorCacheInvalidateCallback,
+									  (Datum) 0);
+		relation_callbacks_registered = true;
+	}
+
 	buffer->by_txn_last_xid = InvalidTransactionId;
 	buffer->by_txn_last_txn = NULL;
 
@@ -372,6 +473,8 @@ ReorderBufferAllocate(void)
 	/* txn_heap is ordered by transaction size */
 	buffer->txn_heap = pairingheap_allocate(ReorderBufferTXNSizeCompare, NULL);
 
+	buffer->can_filter_change = true;
+
 	buffer->spillTxns = 0;
 	buffer->spillCount = 0;
 	buffer->spillBytes = 0;
@@ -826,6 +929,14 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 
 		toptxn->txn_flags |= RBTXN_HAS_STREAMABLE_CHANGE;
 	}
+	else if (change->action == REORDER_BUFFER_CHANGE_INVALIDATION ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID)
+	{
+		ReorderBufferTXN *toptxn = rbtxn_get_toptxn(txn);
+
+		toptxn->txn_flags |= RBTXN_HAS_SNAPSHOT_CHANGES;
+	}
 
 	change->lsn = lsn;
 	change->txn = txn;
@@ -835,6 +946,17 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 	txn->nentries++;
 	txn->nentries_mem++;
 
+	/*
+	 * If filtering is currently suspended and we've crossed the change threshold,
+	 * attempt to filter again
+	 */
+	if (!rb->can_filter_change && (++rb->unfiltered_changes_count
+									>= CHANGES_THRESHOLD_FOR_FILTER))
+	{
+		rb->can_filter_change = true;
+		rb->unfiltered_changes_count = 0;
+	}
+
 	/* update memory accounting information */
 	ReorderBufferChangeMemoryUpdate(rb, change, NULL, true,
 									ReorderBufferChangeSize(change));
@@ -1734,6 +1856,9 @@ ReorderBufferTruncateTXN(ReorderBuffer *rb, ReorderBufferTXN *txn, bool txn_prep
 		txn->txn_flags |= RBTXN_IS_SERIALIZED_CLEAR;
 	}
 
+	/* All snapshot changes up to this point have been processed. */
+	txn->txn_flags &= ~RBTXN_HAS_SNAPSHOT_CHANGES;
+
 	/* also reset the number of entries in the transaction */
 	txn->nentries_mem = 0;
 	txn->nentries = 0;
@@ -2177,6 +2302,84 @@ ReorderBufferResetTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 }
 
 /*
+ * Return the relation corresponding to the given RelFileLocator 'rlocator' if
+ * relevant to decoding. Otherwise, return NULL.
+ *
+ * We don't decode catalog files, relations that are not logically logged,
+ * temporary heaps and sequences.
+ */
+static Relation
+GetDecodableRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+						 bool has_tuple)
+{
+	bool		filterable = false;
+	Relation	relation;
+	Oid			reloid;
+	RelPathStr  path;
+
+	reloid = RelidByRelfilenumber(rlocator->spcOid, rlocator->relNumber);
+	path = relpathperm(*rlocator, MAIN_FORKNUM);
+
+	if (!OidIsValid(reloid))
+	{
+		if (!has_tuple)
+		{
+			/*
+			 * Mapped catalog tuple without data, emitted while catalog table was in
+			 * the process of being rewritten. We can fail to look up the
+			 * relfilenumber, because the relmapper has no "historic" view, in
+			 * contrast to the normal catalog during decoding. Thus repeated rewrites
+			 * can cause a lookup failure. That's OK because we do not decode catalog
+			 * changes anyway. Normally such tuples would be skipped over below, but
+			 * we can't identify whether the table should be logically logged without
+			 * mapping the relfilenumber to the oid.
+			 */
+			return NULL;
+		}
+
+		elog(ERROR, "could not map filenumber \"%s\" to relation OID",
+				path.str);
+	}
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid, path.str);
+
+	if (!RelationIsLogicallyLogged(relation))
+		filterable = true;
+
+	else if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		/*
+		 * Ignore temporary heaps created during DDL unless the plugin has asked
+		 * for them.
+		 */
+		filterable = true;
+	}
+
+	else if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		/*
+		 * For now ignore sequence changes entirely. Most of the time they don't
+		 * log changes using records we understand, so it doesn't make sense to
+		 * handle the few cases we do.
+		 */
+		filterable = true;
+	}
+
+	/* Return NULL to indicate that this relation need not be decoded. */
+	if (filterable)
+	{
+		RelationClose(relation);
+		return NULL;
+	}
+
+	return relation;
+}
+
+/*
  * Helper function for ReorderBufferReplay and ReorderBufferStreamTXN.
  *
  * Send data of a transaction (and its subtransactions) to the
@@ -2248,7 +2451,6 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 		while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
 		{
 			Relation	relation = NULL;
-			Oid			reloid;
 
 			CHECK_FOR_INTERRUPTS();
 
@@ -2307,55 +2509,10 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 				case REORDER_BUFFER_CHANGE_DELETE:
 					Assert(snapshot_now);
 
-					reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
-												  change->data.tp.rlocator.relNumber);
-
-					/*
-					 * Mapped catalog tuple without data, emitted while
-					 * catalog table was in the process of being rewritten. We
-					 * can fail to look up the relfilenumber, because the
-					 * relmapper has no "historic" view, in contrast to the
-					 * normal catalog during decoding. Thus repeated rewrites
-					 * can cause a lookup failure. That's OK because we do not
-					 * decode catalog changes anyway. Normally such tuples
-					 * would be skipped over below, but we can't identify
-					 * whether the table should be logically logged without
-					 * mapping the relfilenumber to the oid.
-					 */
-					if (reloid == InvalidOid &&
-						change->data.tp.newtuple == NULL &&
-						change->data.tp.oldtuple == NULL)
-						goto change_done;
-					else if (reloid == InvalidOid)
-						elog(ERROR, "could not map filenumber \"%s\" to relation OID",
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM).str);
-
-					relation = RelationIdGetRelation(reloid);
-
+					relation = GetDecodableRelation(rb, &change->data.tp.rlocator,
+														(change->data.tp.newtuple != NULL ||
+														 change->data.tp.oldtuple != NULL));
 					if (!RelationIsValid(relation))
-						elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
-							 reloid,
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM).str);
-
-					if (!RelationIsLogicallyLogged(relation))
-						goto change_done;
-
-					/*
-					 * Ignore temporary heaps created during DDL unless the
-					 * plugin has asked for them.
-					 */
-					if (relation->rd_rel->relrewrite && !rb->output_rewrites)
-						goto change_done;
-
-					/*
-					 * For now ignore sequence changes entirely. Most of the
-					 * time they don't log changes using records we
-					 * understand, so it doesn't make sense to handle the few
-					 * cases we do.
-					 */
-					if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
 						goto change_done;
 
 					/* user-triggered change */
@@ -4171,19 +4328,13 @@ ReorderBufferCanStartStreaming(ReorderBuffer *rb)
 }
 
 /*
- * Send data of a large transaction (and its subtransactions) to the
- * output plugin, but using the stream API.
+ * This function generates or retrieves a consistent snapshot of a transaction
+ * that is currently in progress for logical replication.
  */
-static void
-ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+static Snapshot
+ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb, ReorderBufferTXN *txn)
 {
 	Snapshot	snapshot_now;
-	CommandId	command_id;
-	Size		stream_bytes;
-	bool		txn_is_streamed;
-
-	/* We can never reach here for a subtransaction. */
-	Assert(rbtxn_is_toptxn(txn));
 
 	/*
 	 * We can't make any assumptions about base snapshot here, similar to what
@@ -4226,12 +4377,11 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		if (txn->base_snapshot == NULL)
 		{
 			Assert(txn->ninvalidations == 0);
-			return;
+			return NULL;
 		}
 
-		command_id = FirstCommandId;
 		snapshot_now = ReorderBufferCopySnap(rb, txn->base_snapshot,
-											 txn, command_id);
+											 txn, FirstCommandId);
 	}
 	else
 	{
@@ -4244,17 +4394,40 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		 * the LSN condition in the previous branch (so no need to walk
 		 * through subxacts again). In fact, we must not do that as we may be
 		 * using snapshot half-way through the subxact.
-		 */
-		command_id = txn->command_id;
-
-		/*
+		 *
 		 * We can't use txn->snapshot_now directly because after the last
 		 * streaming run, we might have got some new sub-transactions. So we
 		 * need to add them to the snapshot.
 		 */
 		snapshot_now = ReorderBufferCopySnap(rb, txn->snapshot_now,
-											 txn, command_id);
+											 txn, txn->command_id);
+	}
+
+	return snapshot_now;
+}
+
+/*
+ * Send data of a large transaction (and its subtransactions) to the
+ * output plugin, but using the stream API.
+ */
+static void
+ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+	Snapshot	snapshot_now;
+	Size		stream_bytes;
+	bool		txn_is_streamed;
+
+	/* We can never reach here for a subtransaction. */
+	Assert(rbtxn_is_toptxn(txn));
+
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, txn);
+
+	/* This transaction didn't make any changes to the database till now. */
+	if (snapshot_now == NULL)
+		return;
 
+	if (txn->snapshot_now != NULL)
+	{
 		/* Free the previously copied snapshot. */
 		Assert(txn->snapshot_now->copied);
 		ReorderBufferFreeSnap(rb, txn->snapshot_now);
@@ -4272,7 +4445,7 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 
 	/* Process and send the changes to output plugin. */
 	ReorderBufferProcessTXN(rb, txn, InvalidXLogRecPtr, snapshot_now,
-							command_id, true);
+							snapshot_now->curcid, true);
 
 	rb->streamCount += 1;
 	rb->streamBytes += stream_bytes;
@@ -5461,3 +5634,159 @@ restart:
 		*cmax = ent->cmax;
 	return true;
 }
+
+/*
+ * Flush mapping entries when the corresponding relations are updated in pg_class.
+ * If relid is InvalidOid then do a complete reset of the RelFileLocatorFilterCache.
+ */
+static void
+RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid)
+{
+	HASH_SEQ_STATUS status;
+	ReorderBufferRelFileLocatorEnt *entry;
+
+	if (!RelFileLocatorFilterCache)
+		return;
+
+	hash_seq_init(&status, RelFileLocatorFilterCache);
+
+	/* Slightly inefficient algorithm but not performance critical path */
+	while ((entry = (ReorderBufferRelFileLocatorEnt *) hash_seq_search(&status)) != NULL)
+	{
+		/*
+		 * If a complete reset is requested (when relid parameter is InvalidOid),
+		 * we must remove all entries, otherwise just remove the specific relation's
+		 * entry. Remove any invalid cache entries (these are indicated by invalid
+		 * entry->relid)
+		 */
+		if (relid == InvalidOid ||	/* complete reset */
+			entry->relid == InvalidOid ||	/* invalid cache entry */
+			entry->relid == relid)	/* individual flushed relation */
+		{
+			if (hash_search(RelFileLocatorFilterCache,
+							&entry->key,
+							HASH_REMOVE,
+							NULL) == NULL)
+				elog(ERROR, "hash table corrupted");
+		}
+	}
+}
+
+/*
+ * Context reset/delete callback for RelFileLocatorFilterCache.
+ */
+static void
+ReorderBufferMemoryResetcallback(void *arg)
+{
+	RelFileLocatorFilterCache = NULL;
+}
+
+/*
+ * Determine whether the record corresponding to the relation identified by
+ * 'rlocator' needs to be filtered and not decoded and queued in the buffer.
+ *
+ * Determining the necessity of decoding requires accessing relation
+ * information from the system catalog, which requires a historical snapshot.
+ * We cannot directly use the current snapshot of the transaction due to the
+ * presence of INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION records in the
+ * buffer that could modify the snapshot. A proper snapshot can only be
+ * constructed after these records are processed in ReorderBufferProcessTXN, or
+ * if we are decoding a transaction without these records. See comment on top
+ * of GetDecodableRelation() to see list of relations that are not
+ * decoded by the reorderbuffer.
+ *
+ * To reduce the overhead from the system catalog access and transaction
+ * management, the results are cached in the 'RelFileLocatorFilterCache' hash
+ * table.
+ *
+ * Returns true if the relation can be filtered; otherwise, false.
+ */
+bool
+ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+									XLogRecPtr lsn, RelFileLocator *rlocator)
+
+{
+	bool		found;
+	Relation	relation;
+	bool		using_subtxn;
+	Snapshot	snapshot_now;
+	ReorderBufferTXN *txn,
+			   *toptxn;
+	ReorderBufferRelFileLocatorEnt *entry;
+	ReorderBufferRelFileLocatorKey key;
+
+	Assert(RelFileLocatorFilterCache);
+
+	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+	Assert(txn);
+	toptxn = rbtxn_get_toptxn(txn);
+	rb->can_filter_change = false;
+
+	/*
+	 * We cannot construct an accurate historical snapshot until all pending
+	 * records in this transaction that might update the snapshot are
+	 * processed.
+	 */
+	if (rbtxn_has_snapshot_changes(toptxn))
+		return false;
+
+	key.reltablespace = rlocator->spcOid;
+	key.relfilenumber = rlocator->relNumber;
+	entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+	if (found)
+	{
+		rb->can_filter_change = entry->filterable;
+		return entry->filterable;
+	}
+
+	/* constructs a temporary historical snapshot */
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, toptxn);
+	Assert(snapshot_now);
+
+	/* build data to be able to lookup the CommandIds of catalog tuples */
+	ReorderBufferBuildTupleCidHash(rb, toptxn);
+
+	/* setup the initial snapshot */
+	SetupHistoricSnapshot(snapshot_now, toptxn->tuplecid_hash);
+
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by RelFileLocator");
+	else
+		StartTransactionCommand();
+
+	relation = GetDecodableRelation(rb, rlocator, true);
+
+	if (RelationIsValid(relation))
+	{
+		entry->relid = RelationGetRelid(relation);
+		entry->filterable = false;
+		RelationClose(relation);
+	}
+	else
+	{
+		entry->relid = InvalidOid;
+		entry->filterable = true;
+	}
+
+	rb->can_filter_change = entry->filterable;
+
+	ReorderBufferFreeSnap(rb, snapshot_now);
+
+	TeardownHistoricSnapshot(false);
+
+	/*
+	 * Aborting the current (sub-)transaction as a whole has the right
+	 * semantics. We want all locks acquired in here to be released, not
+	 * reassigned to the parent and we do not want any database access have
+	 * persistent effects.
+	 */
+	AbortCurrentTransaction();
+
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	return entry->filterable;
+}
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 517a8e3..24166c2 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -176,6 +176,7 @@ typedef struct ReorderBufferChange
 #define RBTXN_SENT_PREPARE			0x0200
 #define RBTXN_IS_COMMITTED			0x0400
 #define RBTXN_IS_ABORTED			0x0800
+#define RBTXN_HAS_SNAPSHOT_CHANGES  0x1000
 
 #define RBTXN_PREPARE_STATUS_MASK	(RBTXN_IS_PREPARED | RBTXN_SKIPPED_PREPARE | RBTXN_SENT_PREPARE)
 
@@ -215,6 +216,12 @@ typedef struct ReorderBufferChange
 	((txn)->txn_flags & RBTXN_HAS_STREAMABLE_CHANGE) != 0 \
 )
 
+/* Does this transaction make changes to the current snapshot? */
+#define rbtxn_has_snapshot_changes(txn) \
+( \
+	((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
+)
+
 /*
  * Has this transaction been streamed to downstream?
  *
@@ -641,6 +648,7 @@ struct ReorderBuffer
 	 * Private memory context.
 	 */
 	MemoryContext context;
+	MemoryContextCallback relfile_callback;
 
 	/*
 	 * Memory contexts for specific types objects
@@ -661,6 +669,12 @@ struct ReorderBuffer
 	/* Max-heap for sizes of all top-level and sub transactions */
 	pairingheap *txn_heap;
 
+	/* should we try to filter the change? */
+	bool		can_filter_change;
+
+	/* number of changes after a failed attempt at filtering */
+	int8		unfiltered_changes_count;
+
 	/*
 	 * Statistics about transactions spilled to disk.
 	 *
@@ -761,4 +775,8 @@ extern void ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr);
 
 extern void StartupReorderBuffer(void);
 
+extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+												XLogRecPtr lsn, RelFileLocator *rlocator);
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
+
 #endif
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 56989aa..fc12839 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2449,6 +2449,8 @@ ReorderBufferIterTXNEntry
 ReorderBufferIterTXNState
 ReorderBufferMessageCB
 ReorderBufferPrepareCB
+ReorderBufferRelFileLocatorEnt
+ReorderBufferRelFileLocatorKey
 ReorderBufferRollbackPreparedCB
 ReorderBufferStreamAbortCB
 ReorderBufferStreamChangeCB
-- 
1.8.3.1

#35Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#34)
1 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Mon, Mar 3, 2025 at 9:10 PM Ajin Cherian <itsajin@gmail.com> wrote:

On Thu, Feb 20, 2025 at 8:42 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:

I confirmed with tests that with the patch, a lot less disk space is
used when there are lots of unpublished changes and the subscription
is not configured to be streaming.
HEAD code:
Initial disk usage in replication slot directory:
4.0K /home/ajin/dataoss/pg_replslot/mysub
BEGIN
INSERT 0 1000000
COMMIT
Final disk usage in replication slot directory:
212M /home/ajin/dataoss/pg_replslot/mysub

Patched code:
Initial disk usage in replication slot directory:
4.0K /home/ajin/dataoss/pg_replslot/mysub
BEGIN
INSERT 0 1000000
COMMIT
Final disk usage in replication slot directory:
4.0K /home/ajin/dataoss/pg_replslot/mysub

Attaching the test script used.

Regards,
Ajin Cherian
Fujitsu Australia

Attachments:

test_disk_usage.shapplication/octet-stream; name=test_disk_usage.shDownload
#36Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#35)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Moving this patch to the next CF as this patch needs more design level
inputs which may not be feasible in this CF but do continue to review
the patch.

regards,
Ajin Cherian
Fujitsu Australia

#37Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#36)
3 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Thu, Mar 13, 2025 at 5:49 PM Ajin Cherian <itsajin@gmail.com> wrote:

Moving this patch to the next CF as this patch needs more design level
inputs which may not be feasible in this CF but do continue to review
the patch.

regards,
Ajin Cherian
Fujitsu Australia

Rebased the patch as it no longer applied.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

v16-0001-Filter-transactions-that-need-not-be-published.patchapplication/octet-stream; name=v16-0001-Filter-transactions-that-need-not-be-published.patchDownload
From 58217e3a722358b4d46ce56a427e1e081b7b7451 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Thu, 20 Mar 2025 08:22:32 -0400
Subject: [PATCH v16 1/3] Filter transactions that need not be published.

This patch set aims to filter transactions at the decode stage rather than
streaming time, which allows the system to skip processing transactions that do
not contain tables included in the logical replication walsender's publication
list. As a result, this can prevent the serialization of unnecessary records to
disk during non-streaming mode, especially in large transactions, and also
reduce memory and CPU usage in streaming mode when many changes can be filtered
out.

A hash cache of relation file locators is implemented to cache whether a
relation is filterable or not. This ensures that we only need to retrieve
relation and publication information from the catalog when a cache entry is
invalid, avoiding the overhead of starting a transaction for each record.

Filtering is temporarily suspended for a sequence of changes (set to 100
changes) when an unfilterable change is encountered. This strategy minimizes
the overhead of hash searching for every record, which is beneficial when the
majority of tables in an instance are published and thus unfilterable. The
threshold of 100 was determined to be the optimal balance based on performance
tests.

Additionally, filtering is paused for transactions containing WAL records
(INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION) that modify the historical
snapshot constructed during logical decoding. This pause is necessary because
constructing a correct historical snapshot for searching publication
information requires processing these WAL records.

Note that this patch filters changes only for system catalog relations,
non-logically logged relations, or temporary heaps and sequences. A subsequent
patch will introduce a new output plugin in pgoutput, which will further filter
changes for relations not included in publications.
---
 src/backend/replication/logical/decode.c        |   5 +
 src/backend/replication/logical/reorderbuffer.c | 467 ++++++++++++++++++++----
 src/include/replication/reorderbuffer.h         |  18 +
 src/tools/pgindent/typedefs.list                |   2 +
 4 files changed, 423 insertions(+), 69 deletions(-)

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 78f9a0a..30b4d33 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	if (ctx->reorder->can_filter_change &&
+			ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+												buf->origptr, &target_locator))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	if (!(xlrec->flags & XLH_INSERT_IS_SPECULATIVE))
 		change->action = REORDER_BUFFER_CHANGE_INSERT;
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 977fbcd..3cf3030 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -82,6 +82,32 @@
  *	  a bit more memory to the oldest subtransactions, because it's likely
  *	  they are the source for the next sequence of changes.
  *
+ *	  We also try to filter changes at the decode stage rather than at
+ *	  streaming time, which allows the system to skip processing txns that
+ *	  do not contain tables that need to be decoded. As a result,
+ *	  this can prevent the serialization of unnecessary records to disk
+ *	  during non-streaming mode, especially in large transactions, and also
+ *	  reduce memory and CPU usage in streaming mode when many changes can be
+ *	  filtered out.
+ *
+ *	  A hash cache of relation file locators is implemented to cache whether a
+ *	  relation is filterable or not. This ensures that we only need to retrieve
+ *	  relation and publication information from the catalog when a cache entry is
+ *	  invalid, avoiding the overhead of starting a transaction for each record.
+ *
+ *	  Filtering is temporarily suspended for a sequence of changes (set to 100
+ *	  changes) when an unfilterable change is encountered. This strategy minimizes
+ *	  the overhead of hash searching for every record, which is beneficial when the
+ *	  majority of tables in an instance are published and thus unfilterable. The
+ *	  threshold of 100 was determined to be the optimal balance based on performance
+ *	  tests.
+ *
+ *	  Additionally, filtering is paused for transactions containing WAL records
+ *	  (INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION) that modify the historical
+ *	  snapshot constructed during logical decoding. This pause is necessary because
+ *	  constructing a correct historical snapshot for searching publication
+ *	  information requires processing these WAL records.
+ *
  * -------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -109,6 +135,7 @@
 #include "storage/procarray.h"
 #include "storage/sinval.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 #include "utils/relfilenumbermap.h"
@@ -227,8 +254,10 @@ static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
 											   XLogRecPtr lsn, bool create_as_top);
 static void ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
 											  ReorderBufferTXN *subtxn);
-
 static void AssertTXNLsnOrder(ReorderBuffer *rb);
+static Relation GetDecodableRelation(ReorderBuffer *rb,
+										 RelFileLocator *rlocator,
+										 bool has_tuple);
 
 /* ---------------------------------------
  * support functions for lsn-order iterating over the ->changes of a
@@ -279,6 +308,8 @@ static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
  */
 static inline bool ReorderBufferCanStream(ReorderBuffer *rb);
 static inline bool ReorderBufferCanStartStreaming(ReorderBuffer *rb);
+static Snapshot ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb,
+											   ReorderBufferTXN *txn);
 static void ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
 static void ReorderBufferStreamCommit(ReorderBuffer *rb, ReorderBufferTXN *txn);
 
@@ -305,6 +336,48 @@ static void ReorderBufferChangeMemoryUpdate(ReorderBuffer *rb,
 											bool addition, Size sz);
 
 /*
+ * ---------------------------------------
+ * RelFileLocator filtering
+ * ---------------------------------------
+ * This hash table serves as a lookup table for determining if a relation can
+ * be filtered before being decoded and queued into the buffer.
+ *
+ * The hash table shares the same lifespan as the reorder buffer. This is
+ * crucial because each reorderbuffer may have different configurations or be
+ * associated with different output plugins, affecting the types of tables to
+ * be processed.
+ */
+
+static HTAB *RelFileLocatorFilterCache = NULL;
+
+static bool relation_callbacks_registered = false;
+
+typedef struct ReorderBufferRelFileLocatorKey
+{
+	Oid			reltablespace;
+	RelFileNumber relfilenumber;
+} ReorderBufferRelFileLocatorKey;
+
+/* Hash table entry used to determine if the relation can be filtered. */
+typedef struct ReorderBufferRelFileLocatorEnt
+{
+	ReorderBufferRelFileLocatorKey key;
+	Oid			relid;
+	bool		filterable;
+} ReorderBufferRelFileLocatorEnt;
+
+static void RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid);
+static void ReorderBufferMemoryResetcallback(void *arg);
+
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100
+
+/*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
  * prior ReorderBuffer instances for the same slot.
  */
@@ -362,6 +435,34 @@ ReorderBufferAllocate(void)
 	buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
 								 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
+	/*
+	 * To support early filtering of changes, this hash table acts
+	 * as a lookup table to determine if the corresponding relation is
+	 * required to be decoded and queued into the buffer. The hash table
+	 * shares the same lifespan as the reorder buffer.
+	 *
+	 * Also setup the callback to invalidate cache when relations are updated.
+	 */
+	hash_ctl.keysize = sizeof(ReorderBufferRelFileLocatorKey);
+	hash_ctl.entrysize = sizeof(ReorderBufferRelFileLocatorEnt);
+	hash_ctl.hcxt = buffer->context;
+
+	RelFileLocatorFilterCache =
+		hash_create("RelFileLocatorFilterCache", 1000, &hash_ctl,
+					HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	buffer->relfile_callback.arg = NULL;
+	buffer->relfile_callback.func = ReorderBufferMemoryResetcallback;
+	MemoryContextRegisterResetCallback(buffer->context, &buffer->relfile_callback);
+
+	if (!relation_callbacks_registered)
+	{
+		/* Watch for invalidation events. */
+		CacheRegisterRelcacheCallback(RelFileLocatorCacheInvalidateCallback,
+									  (Datum) 0);
+		relation_callbacks_registered = true;
+	}
+
 	buffer->by_txn_last_xid = InvalidTransactionId;
 	buffer->by_txn_last_txn = NULL;
 
@@ -372,6 +473,8 @@ ReorderBufferAllocate(void)
 	/* txn_heap is ordered by transaction size */
 	buffer->txn_heap = pairingheap_allocate(ReorderBufferTXNSizeCompare, NULL);
 
+	buffer->can_filter_change = true;
+
 	buffer->spillTxns = 0;
 	buffer->spillCount = 0;
 	buffer->spillBytes = 0;
@@ -826,6 +929,14 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 
 		toptxn->txn_flags |= RBTXN_HAS_STREAMABLE_CHANGE;
 	}
+	else if (change->action == REORDER_BUFFER_CHANGE_INVALIDATION ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID)
+	{
+		ReorderBufferTXN *toptxn = rbtxn_get_toptxn(txn);
+
+		toptxn->txn_flags |= RBTXN_HAS_SNAPSHOT_CHANGES;
+	}
 
 	change->lsn = lsn;
 	change->txn = txn;
@@ -835,6 +946,17 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 	txn->nentries++;
 	txn->nentries_mem++;
 
+	/*
+	 * If filtering is currently suspended and we've crossed the change threshold,
+	 * attempt to filter again
+	 */
+	if (!rb->can_filter_change && (++rb->unfiltered_changes_count
+									>= CHANGES_THRESHOLD_FOR_FILTER))
+	{
+		rb->can_filter_change = true;
+		rb->unfiltered_changes_count = 0;
+	}
+
 	/* update memory accounting information */
 	ReorderBufferChangeMemoryUpdate(rb, change, NULL, true,
 									ReorderBufferChangeSize(change));
@@ -1734,6 +1856,9 @@ ReorderBufferTruncateTXN(ReorderBuffer *rb, ReorderBufferTXN *txn, bool txn_prep
 		txn->txn_flags |= RBTXN_IS_SERIALIZED_CLEAR;
 	}
 
+	/* All snapshot changes up to this point have been processed. */
+	txn->txn_flags &= ~RBTXN_HAS_SNAPSHOT_CHANGES;
+
 	/* also reset the number of entries in the transaction */
 	txn->nentries_mem = 0;
 	txn->nentries = 0;
@@ -2177,6 +2302,84 @@ ReorderBufferResetTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 }
 
 /*
+ * Return the relation corresponding to the given RelFileLocator 'rlocator' if
+ * relevant to decoding. Otherwise, return NULL.
+ *
+ * We don't decode catalog files, relations that are not logically logged,
+ * temporary heaps and sequences.
+ */
+static Relation
+GetDecodableRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+						 bool has_tuple)
+{
+	bool		filterable = false;
+	Relation	relation;
+	Oid			reloid;
+	RelPathStr  path;
+
+	reloid = RelidByRelfilenumber(rlocator->spcOid, rlocator->relNumber);
+	path = relpathperm(*rlocator, MAIN_FORKNUM);
+
+	if (!OidIsValid(reloid))
+	{
+		if (!has_tuple)
+		{
+			/*
+			 * Mapped catalog tuple without data, emitted while catalog table was in
+			 * the process of being rewritten. We can fail to look up the
+			 * relfilenumber, because the relmapper has no "historic" view, in
+			 * contrast to the normal catalog during decoding. Thus repeated rewrites
+			 * can cause a lookup failure. That's OK because we do not decode catalog
+			 * changes anyway. Normally such tuples would be skipped over below, but
+			 * we can't identify whether the table should be logically logged without
+			 * mapping the relfilenumber to the oid.
+			 */
+			return NULL;
+		}
+
+		elog(ERROR, "could not map filenumber \"%s\" to relation OID",
+				path.str);
+	}
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid, path.str);
+
+	if (!RelationIsLogicallyLogged(relation))
+		filterable = true;
+
+	else if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		/*
+		 * Ignore temporary heaps created during DDL unless the plugin has asked
+		 * for them.
+		 */
+		filterable = true;
+	}
+
+	else if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		/*
+		 * For now ignore sequence changes entirely. Most of the time they don't
+		 * log changes using records we understand, so it doesn't make sense to
+		 * handle the few cases we do.
+		 */
+		filterable = true;
+	}
+
+	/* Return NULL to indicate that this relation need not be decoded. */
+	if (filterable)
+	{
+		RelationClose(relation);
+		return NULL;
+	}
+
+	return relation;
+}
+
+/*
  * Helper function for ReorderBufferReplay and ReorderBufferStreamTXN.
  *
  * Send data of a transaction (and its subtransactions) to the
@@ -2248,7 +2451,6 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 		while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
 		{
 			Relation	relation = NULL;
-			Oid			reloid;
 
 			CHECK_FOR_INTERRUPTS();
 
@@ -2307,55 +2509,10 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 				case REORDER_BUFFER_CHANGE_DELETE:
 					Assert(snapshot_now);
 
-					reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
-												  change->data.tp.rlocator.relNumber);
-
-					/*
-					 * Mapped catalog tuple without data, emitted while
-					 * catalog table was in the process of being rewritten. We
-					 * can fail to look up the relfilenumber, because the
-					 * relmapper has no "historic" view, in contrast to the
-					 * normal catalog during decoding. Thus repeated rewrites
-					 * can cause a lookup failure. That's OK because we do not
-					 * decode catalog changes anyway. Normally such tuples
-					 * would be skipped over below, but we can't identify
-					 * whether the table should be logically logged without
-					 * mapping the relfilenumber to the oid.
-					 */
-					if (reloid == InvalidOid &&
-						change->data.tp.newtuple == NULL &&
-						change->data.tp.oldtuple == NULL)
-						goto change_done;
-					else if (reloid == InvalidOid)
-						elog(ERROR, "could not map filenumber \"%s\" to relation OID",
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM).str);
-
-					relation = RelationIdGetRelation(reloid);
-
+					relation = GetDecodableRelation(rb, &change->data.tp.rlocator,
+														(change->data.tp.newtuple != NULL ||
+														 change->data.tp.oldtuple != NULL));
 					if (!RelationIsValid(relation))
-						elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
-							 reloid,
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM).str);
-
-					if (!RelationIsLogicallyLogged(relation))
-						goto change_done;
-
-					/*
-					 * Ignore temporary heaps created during DDL unless the
-					 * plugin has asked for them.
-					 */
-					if (relation->rd_rel->relrewrite && !rb->output_rewrites)
-						goto change_done;
-
-					/*
-					 * For now ignore sequence changes entirely. Most of the
-					 * time they don't log changes using records we
-					 * understand, so it doesn't make sense to handle the few
-					 * cases we do.
-					 */
-					if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
 						goto change_done;
 
 					/* user-triggered change */
@@ -4171,19 +4328,13 @@ ReorderBufferCanStartStreaming(ReorderBuffer *rb)
 }
 
 /*
- * Send data of a large transaction (and its subtransactions) to the
- * output plugin, but using the stream API.
+ * This function generates or retrieves a consistent snapshot of a transaction
+ * that is currently in progress for logical replication.
  */
-static void
-ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+static Snapshot
+ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb, ReorderBufferTXN *txn)
 {
 	Snapshot	snapshot_now;
-	CommandId	command_id;
-	Size		stream_bytes;
-	bool		txn_is_streamed;
-
-	/* We can never reach here for a subtransaction. */
-	Assert(rbtxn_is_toptxn(txn));
 
 	/*
 	 * We can't make any assumptions about base snapshot here, similar to what
@@ -4226,12 +4377,11 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		if (txn->base_snapshot == NULL)
 		{
 			Assert(txn->ninvalidations == 0);
-			return;
+			return NULL;
 		}
 
-		command_id = FirstCommandId;
 		snapshot_now = ReorderBufferCopySnap(rb, txn->base_snapshot,
-											 txn, command_id);
+											 txn, FirstCommandId);
 	}
 	else
 	{
@@ -4244,17 +4394,40 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		 * the LSN condition in the previous branch (so no need to walk
 		 * through subxacts again). In fact, we must not do that as we may be
 		 * using snapshot half-way through the subxact.
-		 */
-		command_id = txn->command_id;
-
-		/*
+		 *
 		 * We can't use txn->snapshot_now directly because after the last
 		 * streaming run, we might have got some new sub-transactions. So we
 		 * need to add them to the snapshot.
 		 */
 		snapshot_now = ReorderBufferCopySnap(rb, txn->snapshot_now,
-											 txn, command_id);
+											 txn, txn->command_id);
+	}
+
+	return snapshot_now;
+}
+
+/*
+ * Send data of a large transaction (and its subtransactions) to the
+ * output plugin, but using the stream API.
+ */
+static void
+ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+	Snapshot	snapshot_now;
+	Size		stream_bytes;
+	bool		txn_is_streamed;
+
+	/* We can never reach here for a subtransaction. */
+	Assert(rbtxn_is_toptxn(txn));
+
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, txn);
+
+	/* This transaction didn't make any changes to the database till now. */
+	if (snapshot_now == NULL)
+		return;
 
+	if (txn->snapshot_now != NULL)
+	{
 		/* Free the previously copied snapshot. */
 		Assert(txn->snapshot_now->copied);
 		ReorderBufferFreeSnap(rb, txn->snapshot_now);
@@ -4272,7 +4445,7 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 
 	/* Process and send the changes to output plugin. */
 	ReorderBufferProcessTXN(rb, txn, InvalidXLogRecPtr, snapshot_now,
-							command_id, true);
+							snapshot_now->curcid, true);
 
 	rb->streamCount += 1;
 	rb->streamBytes += stream_bytes;
@@ -5460,3 +5633,159 @@ restart:
 		*cmax = ent->cmax;
 	return true;
 }
+
+/*
+ * Flush mapping entries when the corresponding relations are updated in pg_class.
+ * If relid is InvalidOid then do a complete reset of the RelFileLocatorFilterCache.
+ */
+static void
+RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid)
+{
+	HASH_SEQ_STATUS status;
+	ReorderBufferRelFileLocatorEnt *entry;
+
+	if (!RelFileLocatorFilterCache)
+		return;
+
+	hash_seq_init(&status, RelFileLocatorFilterCache);
+
+	/* Slightly inefficient algorithm but not performance critical path */
+	while ((entry = (ReorderBufferRelFileLocatorEnt *) hash_seq_search(&status)) != NULL)
+	{
+		/*
+		 * If a complete reset is requested (when relid parameter is InvalidOid),
+		 * we must remove all entries, otherwise just remove the specific relation's
+		 * entry. Remove any invalid cache entries (these are indicated by invalid
+		 * entry->relid)
+		 */
+		if (relid == InvalidOid ||	/* complete reset */
+			entry->relid == InvalidOid ||	/* invalid cache entry */
+			entry->relid == relid)	/* individual flushed relation */
+		{
+			if (hash_search(RelFileLocatorFilterCache,
+							&entry->key,
+							HASH_REMOVE,
+							NULL) == NULL)
+				elog(ERROR, "hash table corrupted");
+		}
+	}
+}
+
+/*
+ * Context reset/delete callback for RelFileLocatorFilterCache.
+ */
+static void
+ReorderBufferMemoryResetcallback(void *arg)
+{
+	RelFileLocatorFilterCache = NULL;
+}
+
+/*
+ * Determine whether the record corresponding to the relation identified by
+ * 'rlocator' needs to be filtered and not decoded and queued in the buffer.
+ *
+ * Determining the necessity of decoding requires accessing relation
+ * information from the system catalog, which requires a historical snapshot.
+ * We cannot directly use the current snapshot of the transaction due to the
+ * presence of INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION records in the
+ * buffer that could modify the snapshot. A proper snapshot can only be
+ * constructed after these records are processed in ReorderBufferProcessTXN, or
+ * if we are decoding a transaction without these records. See comment on top
+ * of GetDecodableRelation() to see list of relations that are not
+ * decoded by the reorderbuffer.
+ *
+ * To reduce the overhead from the system catalog access and transaction
+ * management, the results are cached in the 'RelFileLocatorFilterCache' hash
+ * table.
+ *
+ * Returns true if the relation can be filtered; otherwise, false.
+ */
+bool
+ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+									XLogRecPtr lsn, RelFileLocator *rlocator)
+
+{
+	bool		found;
+	Relation	relation;
+	bool		using_subtxn;
+	Snapshot	snapshot_now;
+	ReorderBufferTXN *txn,
+			   *toptxn;
+	ReorderBufferRelFileLocatorEnt *entry;
+	ReorderBufferRelFileLocatorKey key;
+
+	Assert(RelFileLocatorFilterCache);
+
+	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+	Assert(txn);
+	toptxn = rbtxn_get_toptxn(txn);
+	rb->can_filter_change = false;
+
+	/*
+	 * We cannot construct an accurate historical snapshot until all pending
+	 * records in this transaction that might update the snapshot are
+	 * processed.
+	 */
+	if (rbtxn_has_snapshot_changes(toptxn))
+		return false;
+
+	key.reltablespace = rlocator->spcOid;
+	key.relfilenumber = rlocator->relNumber;
+	entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+	if (found)
+	{
+		rb->can_filter_change = entry->filterable;
+		return entry->filterable;
+	}
+
+	/* constructs a temporary historical snapshot */
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, toptxn);
+	Assert(snapshot_now);
+
+	/* build data to be able to lookup the CommandIds of catalog tuples */
+	ReorderBufferBuildTupleCidHash(rb, toptxn);
+
+	/* setup the initial snapshot */
+	SetupHistoricSnapshot(snapshot_now, toptxn->tuplecid_hash);
+
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by RelFileLocator");
+	else
+		StartTransactionCommand();
+
+	relation = GetDecodableRelation(rb, rlocator, true);
+
+	if (RelationIsValid(relation))
+	{
+		entry->relid = RelationGetRelid(relation);
+		entry->filterable = false;
+		RelationClose(relation);
+	}
+	else
+	{
+		entry->relid = InvalidOid;
+		entry->filterable = true;
+	}
+
+	rb->can_filter_change = entry->filterable;
+
+	ReorderBufferFreeSnap(rb, snapshot_now);
+
+	TeardownHistoricSnapshot(false);
+
+	/*
+	 * Aborting the current (sub-)transaction as a whole has the right
+	 * semantics. We want all locks acquired in here to be released, not
+	 * reassigned to the parent and we do not want any database access have
+	 * persistent effects.
+	 */
+	AbortCurrentTransaction();
+
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	return entry->filterable;
+}
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 3be0cbd..e0dfdc8 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -176,6 +176,7 @@ typedef struct ReorderBufferChange
 #define RBTXN_SENT_PREPARE			0x0200
 #define RBTXN_IS_COMMITTED			0x0400
 #define RBTXN_IS_ABORTED			0x0800
+#define RBTXN_HAS_SNAPSHOT_CHANGES  0x1000
 
 #define RBTXN_PREPARE_STATUS_MASK	(RBTXN_IS_PREPARED | RBTXN_SKIPPED_PREPARE | RBTXN_SENT_PREPARE)
 
@@ -215,6 +216,12 @@ typedef struct ReorderBufferChange
 	((txn)->txn_flags & RBTXN_HAS_STREAMABLE_CHANGE) != 0 \
 )
 
+/* Does this transaction make changes to the current snapshot? */
+#define rbtxn_has_snapshot_changes(txn) \
+( \
+	((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
+)
+
 /*
  * Has this transaction been streamed to downstream?
  *
@@ -641,6 +648,7 @@ struct ReorderBuffer
 	 * Private memory context.
 	 */
 	MemoryContext context;
+	MemoryContextCallback relfile_callback;
 
 	/*
 	 * Memory contexts for specific types objects
@@ -661,6 +669,12 @@ struct ReorderBuffer
 	/* Max-heap for sizes of all top-level and sub transactions */
 	pairingheap *txn_heap;
 
+	/* should we try to filter the change? */
+	bool		can_filter_change;
+
+	/* number of changes after a failed attempt at filtering */
+	int8		unfiltered_changes_count;
+
 	/*
 	 * Statistics about transactions spilled to disk.
 	 *
@@ -760,4 +774,8 @@ extern void ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr);
 
 extern void StartupReorderBuffer(void);
 
+extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+												XLogRecPtr lsn, RelFileLocator *rlocator);
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
+
 #endif
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index bfa276d..361ea68 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2482,6 +2482,8 @@ ReorderBufferIterTXNEntry
 ReorderBufferIterTXNState
 ReorderBufferMessageCB
 ReorderBufferPrepareCB
+ReorderBufferRelFileLocatorEnt
+ReorderBufferRelFileLocatorKey
 ReorderBufferRollbackPreparedCB
 ReorderBufferStreamAbortCB
 ReorderBufferStreamChangeCB
-- 
1.8.3.1

v16-0003-Tests-for-filtering-unpublished-changes.patchapplication/octet-stream; name=v16-0003-Tests-for-filtering-unpublished-changes.patchDownload
From 8efb58f9f70b3dff4226b48ffb68bde01f142798 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Thu, 20 Mar 2025 08:30:26 -0400
Subject: [PATCH v16 3/3] Tests for filtering unpublished changes

---
 src/backend/replication/logical/reorderbuffer.c |  2 +-
 src/test/subscription/t/001_rep_changes.pl      | 65 +++++++++++++++++++++++++
 2 files changed, 66 insertions(+), 1 deletion(-)
 mode change 100644 => 100755 src/test/subscription/t/001_rep_changes.pl

diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 787daaa..6ff3656 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -376,7 +376,7 @@ static void ReorderBufferMemoryResetcallback(void *arg);
  * This strategy helps to minimize the overhead of performing a hash table
  * search for each record, especially when most changes are not filterable.
  */
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0
 
 /*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
diff --git a/src/test/subscription/t/001_rep_changes.pl b/src/test/subscription/t/001_rep_changes.pl
old mode 100644
new mode 100755
index 8726fe0..c49aff4
--- a/src/test/subscription/t/001_rep_changes.pl
+++ b/src/test/subscription/t/001_rep_changes.pl
@@ -555,6 +555,71 @@ $result = $node_subscriber->safe_psql('postgres',
 	"SELECT count(*) FROM pg_replication_origin");
 is($result, qq(0), 'check replication origin was dropped on subscriber');
 
+$node_publisher->stop('fast');
+
+# Additional tests for filtering of unpublished changes
+# Bump up log verbosity to DEBUG1 for confirmation logs
+$node_publisher->append_conf('postgresql.conf', "log_min_messages = debug1");
+$node_publisher->start;
+
+# Create new tables on publisher and subscriber
+$node_publisher->safe_psql('postgres', "CREATE TABLE pub_table (id int primary key, data text)");
+$node_publisher->safe_psql('postgres', "CREATE TABLE unpub_table (id int primary key, data text)");
+$node_publisher->safe_psql('postgres', "CREATE TABLE insert_only_table (id int primary key, data text)");
+$node_publisher->safe_psql('postgres', "CREATE TABLE delete_only_table (id int primary key, data text)");
+
+$node_subscriber->safe_psql('postgres', "CREATE TABLE pub_table (id int primary key, data text)");
+$node_subscriber->safe_psql('postgres', "CREATE TABLE unpub_table (id int primary key, data text)");
+$node_subscriber->safe_psql('postgres', "CREATE TABLE insert_only_table (id int primary key, data text)");
+$node_subscriber->safe_psql('postgres', "CREATE TABLE delete_only_table (id int primary key, data text)");
+
+
+# Setup logical replication publications
+$node_publisher->safe_psql('postgres', "CREATE PUBLICATION pub_all FOR TABLE pub_table");
+$node_publisher->safe_psql('postgres', "CREATE PUBLICATION pub_insert_only FOR TABLE insert_only_table WITH (publish = insert)");
+$node_publisher->safe_psql('postgres', "CREATE PUBLICATION pub_delete_only FOR TABLE delete_only_table WITH (publish = delete)");
+
+# Setup logical replication subscription
+$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION sub_all CONNECTION '$publisher_connstr' PUBLICATION pub_all");
+$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION sub_insert_only CONNECTION '$publisher_connstr' PUBLICATION pub_insert_only");
+$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION sub_delete_only CONNECTION '$publisher_connstr' PUBLICATION pub_delete_only");
+
+# Wait for initial sync
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_all');
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_insert_only');
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_delete_only');
+
+# Insert into an unpublished table (should not be replicated)
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO unpub_table VALUES (1, 'unpublished')");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+
+# Insert, delete, and update tests for restricted publication tables
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO insert_only_table VALUES (1, 'to be inserted')");
+$node_publisher->safe_psql('postgres', "UPDATE insert_only_table SET data = 'updated' WHERE id = 1");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering UPDATE/,
+	'unpublished UPDATE is filtered');
+
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "DELETE FROM insert_only_table WHERE id = 1");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering DELETE/,
+	'unpublished DELETE is filtered');
+
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO delete_only_table VALUES (1, 'to be deleted')");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_all");
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_insert_only");
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_delete_only");
+
 $node_subscriber->stop('fast');
 $node_publisher->stop('fast');
 
-- 
1.8.3.1

v16-0002-Introduce-an-output-plugin-callback-to-filter-ch.patchapplication/octet-stream; name=v16-0002-Introduce-an-output-plugin-callback-to-filter-ch.patchDownload
From 576fd3e086063d667673d4dc3788d9edb981cfbd Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Thu, 20 Mar 2025 08:29:31 -0400
Subject: [PATCH v16 2/3] Introduce an output plugin callback to filter changes

This new output plugin callback provides an option to logical decoding plugins to filter out
changes early. The primary purpose of the callback is to conserve memory and processing cycles
by excluding changes that are not required by output plugins.
---
 doc/src/sgml/logicaldecoding.sgml               | 41 ++++++++++++++-
 src/backend/replication/logical/decode.c        | 66 +++++++++++++++++++++--
 src/backend/replication/logical/logical.c       | 42 +++++++++++++++
 src/backend/replication/logical/reorderbuffer.c | 50 ++++++++++++++----
 src/backend/replication/pgoutput/pgoutput.c     | 70 +++++++++++++++++++++++++
 src/include/replication/output_plugin.h         | 20 +++++++
 src/include/replication/reorderbuffer.h         | 10 +++-
 src/test/subscription/t/013_partition.pl        |  7 +++
 8 files changed, 290 insertions(+), 16 deletions(-)

diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 1c4ae38..a603c47 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -560,6 +560,7 @@ typedef struct OutputPluginCallbacks
     LogicalDecodeCommitCB commit_cb;
     LogicalDecodeMessageCB message_cb;
     LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+    LogicalDecodeFilterChangeCB filter_change_cb;
     LogicalDecodeShutdownCB shutdown_cb;
     LogicalDecodeFilterPrepareCB filter_prepare_cb;
     LogicalDecodeBeginPrepareCB begin_prepare_cb;
@@ -582,8 +583,8 @@ typedef void (*LogicalOutputPluginInit) (struct OutputPluginCallbacks *cb);
      and <function>commit_cb</function> callbacks are required,
      while <function>startup_cb</function>, <function>truncate_cb</function>,
      <function>message_cb</function>, <function>filter_by_origin_cb</function>,
-     and <function>shutdown_cb</function> are optional.
-     If <function>truncate_cb</function> is not set but a
+     <function>shutdown_cb</function>, and <function>filter_change_cb</function>
+     are optional. If <function>truncate_cb</function> is not set but a
      <command>TRUNCATE</command> is to be decoded, the action will be ignored.
     </para>
 
@@ -871,6 +872,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
      </para>
      </sect3>
 
+     <sect3 id="logicaldecoding-output-plugin-filter-change">
+     <title>Change Filter Callback</title>
+
+     <para>
+      The optional <function>filter_change_cb</function> is called before a
+      change record is decoded to determine whether the change can be filtered
+      out.
+<programlisting>
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+                                             Oid relid,
+                                             ReorderBufferChangeType change_type,
+                                             bool in_txn, bool *cache_valid);
+</programlisting>
+      To indicate that decoding can be skipped for the given change
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.
+      The <parameter>in_txn</parameter> parameter indicates whether the
+      callback is invoked within a transaction block.
+      When <parameter>in_txn</parameter> is false, and if making a decision to filter a change requires being inside a
+      transaction block, such as needing access to the catalog, set
+      <parameter>*cache_valid</parameter> to <literal>false</literal>.
+      This ensures that the callback will be reinvoked once a transaction block
+      starts. If a decision can be made immediately, set
+      <parameter>*cache_valid</parameter> to <literal>true</literal>.
+     </para>
+     <para>
+      The primary purpose of this callback function is to optimize memory usage
+      and processing efficiency by filtering out changes that are unnecessary for
+      output plugins. It enables output plugins to selectively process relevant
+      changes. Caching filtering decisions locally is recommended, as it enables
+      the callback to provide cached results without repeatedly initiating
+      transactions or querying the catalog. This approach minimizes overhead
+      and improves efficiency during the decoding process.
+     </para>
+     </sect3>
+
     <sect3 id="logicaldecoding-output-plugin-message">
      <title>Generic Message Callback</title>
 
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 30b4d33..abacfaf 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -576,6 +576,17 @@ FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
 }
 
 /*
+ * Check if filtering changes before decoding is supported and we're not suppressing filter
+ * changes currently.
+ */
+static inline bool
+FilterChangeIsEnabled(LogicalDecodingContext *ctx)
+{
+	return (ctx->callbacks.filter_change_cb != NULL &&
+				ctx->reorder->can_filter_change);
+}
+
+/*
  * Handle rmgr LOGICALMSG_ID records for LogicalDecodingProcessRecord().
  */
 void
@@ -881,6 +892,15 @@ DecodeAbort(LogicalDecodingContext *ctx, XLogRecordBuffer *buf,
 	UpdateDecodingStats(ctx);
 }
 
+/* Function to determine whether to skip the change */
+static bool SkipThisChange(LogicalDecodingContext *ctx, XLogRecPtr origptr, TransactionId xid,
+							RelFileLocator *target_locator, ReorderBufferChangeType change_type)
+{
+	return (FilterChangeIsEnabled(ctx) &&
+			ReorderBufferFilterByRelFileLocator(ctx->reorder, xid, origptr, target_locator,
+												change_type));
+}
+
 /*
  * Parse XLOG_HEAP_INSERT (not MULTI_INSERT!) records into tuplebufs.
  *
@@ -915,9 +935,13 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
-	if (ctx->reorder->can_filter_change &&
-			ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
-												buf->origptr, &target_locator))
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+						REORDER_BUFFER_CHANGE_INSERT))
 		return;
 
 	change = ReorderBufferAllocChange(ctx->reorder);
@@ -970,6 +994,15 @@ DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+					   REORDER_BUFFER_CHANGE_UPDATE))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_UPDATE;
 	change->origin_id = XLogRecGetOrigin(r);
@@ -1036,6 +1069,15 @@ DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+					   REORDER_BUFFER_CHANGE_DELETE))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 
 	if (xlrec->flags & XLH_DELETE_IS_SUPER)
@@ -1139,6 +1181,15 @@ DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 		return;
 
 	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &rlocator,
+					   REORDER_BUFFER_CHANGE_INSERT))
+		return;
+
+	/*
 	 * We know that this multi_insert isn't for a catalog, so the block should
 	 * always have data even if a full-page write of it is taken.
 	 */
@@ -1229,6 +1280,15 @@ DecodeSpecConfirm(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/*
+	 * When filtering changes, determine if the relation associated with the change
+	 * can be skipped. This could be because the relation is unlogged or because
+	 * the plugin has opted to exclude this relation from decoding.
+	 */
+	if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+						REORDER_BUFFER_CHANGE_INSERT))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM;
 	change->origin_id = XLogRecGetOrigin(r);
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 8ea846b..635e1de 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -74,6 +74,9 @@ static void truncate_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 							   XLogRecPtr message_lsn, bool transactional,
 							   const char *prefix, Size message_size, const char *message);
+static bool filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+								  ReorderBufferChangeType change_type, bool in_txn,
+								  bool *cache_valid);
 
 /* streaming callbacks */
 static void stream_start_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
@@ -220,6 +223,7 @@ StartupDecodingContext(List *output_plugin_options,
 	/* wrap output plugin callbacks, so we can add error context information */
 	ctx->reorder->begin = begin_cb_wrapper;
 	ctx->reorder->apply_change = change_cb_wrapper;
+	ctx->reorder->filter_change = filter_change_cb_wrapper;
 	ctx->reorder->apply_truncate = truncate_cb_wrapper;
 	ctx->reorder->commit = commit_cb_wrapper;
 	ctx->reorder->message = message_cb_wrapper;
@@ -1224,6 +1228,44 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+static bool
+filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+					  ReorderBufferChangeType change_type, bool in_txn,
+					  bool *cache_valid)
+{
+	LogicalDecodingContext *ctx = cache->private_data;
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	/* check if the filter change callback is supported */
+	if (ctx->callbacks.filter_change_cb == NULL)
+		return false;
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_change";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_change_cb(ctx, relid, change_type, in_txn, cache_valid);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 3cf3030..787daaa 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -371,7 +371,8 @@ static void ReorderBufferMemoryResetcallback(void *arg);
 
 /*
  * After encountering a change that cannot be filtered out, filtering is
- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
  * This strategy helps to minimize the overhead of performing a hash table
  * search for each record, especially when most changes are not filterable.
  */
@@ -5702,10 +5703,11 @@ ReorderBufferMemoryResetcallback(void *arg)
  */
 bool
 ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
-									XLogRecPtr lsn, RelFileLocator *rlocator)
-
+									XLogRecPtr lsn, RelFileLocator *rlocator,
+									ReorderBufferChangeType change_type)
 {
 	bool		found;
+	bool		cache_valid;
 	Relation	relation;
 	bool		using_subtxn;
 	Snapshot	snapshot_now;
@@ -5719,7 +5721,6 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
 	Assert(txn);
 	toptxn = rbtxn_get_toptxn(txn);
-	rb->can_filter_change = false;
 
 	/*
 	 * We cannot construct an accurate historical snapshot until all pending
@@ -5736,7 +5737,25 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (found)
 	{
 		rb->can_filter_change = entry->filterable;
-		return entry->filterable;
+
+		/*
+		 * Quick return if we already know that the relation is not to be decoded.
+		 * These are for special relations that are unlogged and for sequences
+		 * and catalogs.
+		 */
+		if (entry->filterable)
+			return true;
+
+		/* Allow the output plugin to filter relations */
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  false, &cache_valid);
+
+		/*
+		 * If plugin had the relation ready in cache, the response is valid, else
+		 * we'll need to call the plugin a second time within a transaction.
+		 */
+		if (cache_valid)
+			return rb->can_filter_change;
 	}
 
 	/* constructs a temporary historical snapshot */
@@ -5760,18 +5779,29 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 
 	if (RelationIsValid(relation))
 	{
-		entry->relid = RelationGetRelid(relation);
+		if (IsToastRelation(relation))
+		{
+			char   *toast_name = RelationGetRelationName(relation);
+			int		n PG_USED_FOR_ASSERTS_ONLY;
+
+			n = sscanf(toast_name, "pg_toast_%u", &entry->relid);
+
+			Assert(n == 1);
+		}
+		else
+			entry->relid = RelationGetRelid(relation);
+
 		entry->filterable = false;
+		rb->can_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  true, &cache_valid);
 		RelationClose(relation);
 	}
 	else
 	{
 		entry->relid = InvalidOid;
-		entry->filterable = true;
+		rb->can_filter_change = entry->filterable = true;
 	}
 
-	rb->can_filter_change = entry->filterable;
-
 	ReorderBufferFreeSnap(rb, snapshot_now);
 
 	TeardownHistoricSnapshot(false);
@@ -5787,5 +5817,5 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (using_subtxn)
 		RollbackAndReleaseCurrentSubTransaction();
 
-	return entry->filterable;
+	return rb->can_filter_change;
 }
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index 8357bf8..823d4c7b 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -57,6 +57,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+								   ReorderBufferChangeType change_type, bool in_txn,
+								   bool *cache_valid);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -268,6 +271,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_change_cb = pgoutput_filter_change;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1746,6 +1750,72 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 }
 
 /*
+ * Determine whether a change to the specified relation should be published.
+ *
+ * See the comments atop of LogicalDecodeFilterChangeCB for details.
+ */
+static bool
+pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+					   ReorderBufferChangeType change_type, bool in_txn, bool *cache_valid)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *entry;
+
+	Assert(RelationSyncCache != NULL);
+
+	if (in_txn)
+	{
+		Relation	relation;
+
+		relation = RelationIdGetRelation(relid);
+		entry = get_rel_sync_entry(data, relation);
+		*cache_valid = true;
+	}
+	else
+	{
+		entry = (RelationSyncEntry *) hash_search(RelationSyncCache,
+												  &relid,
+												  HASH_FIND, cache_valid);
+		if (!*cache_valid)
+			return false;
+	}
+
+	/*
+	 * If the pubaction is not supported by this publication then return true to say
+	 * the change for this entry can be skipped.
+	 */
+	switch (change_type)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			if (!entry->pubactions.pubinsert)
+			{
+				elog(DEBUG1, "Filtering INSERT");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			if (!entry->pubactions.pubupdate)
+			{
+				elog(DEBUG1, "Filtering UPDATE");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			if (!entry->pubactions.pubdelete)
+			{
+				elog(DEBUG1, "Filtering DELETE");
+				return true;
+			}
+			break;
+		default:
+			/* allow any other changes that are not explicitly filtered */
+			return false;
+	}
+
+	return false;
+}
+
+/*
  * Shutdown the output plugin.
  *
  * Note, we don't need to clean the data->context, data->cachectx, and
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 8d4d5b71..f82bc1d 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -97,6 +97,25 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
 											   RepOriginId origin_id);
 
 /*
+ * This callback is called before a change record is decoded to determine
+ * whether the change can be filtered out before entering the reorder buffer.
+ *
+ * Typically, the output plugin needs to refer to the system catalog to make
+ * this decision. To enhance efficiency, the plugin should cache the lookup
+ * result for each relation, minimizing catalog access on subsequent calls.
+ *
+ * If the callback is called with 'in_txn' set as false (the reorder
+ * buffer has not started a transaction), the output plugin should set '*cache_valid'
+ * to false to indicate that the result is not available in its internal cache.
+ * If 'in_txn' is true, the plugin can create a cache entry after querying the
+ * catalog.
+ */
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
+/*
  * Called to shutdown an output plugin.
  */
 typedef void (*LogicalDecodeShutdownCB) (struct LogicalDecodingContext *ctx);
@@ -222,6 +241,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterChangeCB filter_change_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index e0dfdc8..3a56de5 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -494,6 +494,12 @@ typedef void (*ReorderBufferMessageCB) (ReorderBuffer *rb,
 										const char *prefix, Size sz,
 										const char *message);
 
+/* filter change callback signature */
+typedef bool (*ReorderBufferFilterChangeCB) (ReorderBuffer *rb,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
 /* begin prepare callback signature */
 typedef void (*ReorderBufferBeginPrepareCB) (ReorderBuffer *rb,
 											 ReorderBufferTXN *txn);
@@ -604,6 +610,7 @@ struct ReorderBuffer
 	 */
 	ReorderBufferBeginCB begin;
 	ReorderBufferApplyChangeCB apply_change;
+	ReorderBufferFilterChangeCB filter_change;
 	ReorderBufferApplyTruncateCB apply_truncate;
 	ReorderBufferCommitCB commit;
 	ReorderBufferMessageCB message;
@@ -775,7 +782,8 @@ extern void ReorderBufferSetRestartPoint(ReorderBuffer *rb, XLogRecPtr ptr);
 extern void StartupReorderBuffer(void);
 
 extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
-												XLogRecPtr lsn, RelFileLocator *rlocator);
+												XLogRecPtr lsn, RelFileLocator *rlocator,
+												ReorderBufferChangeType change_type);
 extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
 
 #endif
diff --git a/src/test/subscription/t/013_partition.pl b/src/test/subscription/t/013_partition.pl
index 61b0cb4..a64b27e 100644
--- a/src/test/subscription/t/013_partition.pl
+++ b/src/test/subscription/t/013_partition.pl
@@ -472,6 +472,13 @@ $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4 (a int PRIMARY KEY)");
 $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4_1 (a int PRIMARY KEY)");
+
+# Ensure that the subscription 'sub2' catches up with the latest changes. This
+# is necessary for the walsender to update its historic snapshot. Otherwise,
+# the walsender might retain an outdated snapshot, potentially preventing it
+# from accessing the newly created publication.
+$node_publisher->wait_for_catchup('sub2');
+
 # Since we specified publish_via_partition_root in pub_all and
 # pub_lower_level, all partition tables use their root tables' identity and
 # schema. We set the list of publications so that the FOR ALL TABLES
-- 
1.8.3.1

#38Shlok Kyal
shlok.kyal.oss@gmail.com
In reply to: Ajin Cherian (#37)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Thu, 20 Mar 2025 at 18:09, Ajin Cherian <itsajin@gmail.com> wrote:

On Thu, Mar 13, 2025 at 5:49 PM Ajin Cherian <itsajin@gmail.com> wrote:

Moving this patch to the next CF as this patch needs more design level
inputs which may not be feasible in this CF but do continue to review
the patch.

regards,
Ajin Cherian
Fujitsu Australia

Rebased the patch as it no longer applied.

Hi Ajin,

I have reviewed patch v16-0001, here are my comments:

1. There are some places where comments are more than 80 columns
window. I think it should be <=80 as per [1].
    a. initial comment in reorderbuffer.c
    + *   Filtering is temporarily suspended for a sequence of changes
(set to 100
    + *   changes) when an unfilterable change is encountered. This
strategy minimizes
    + *   the overhead of hash searching for every record, which is
beneficial when the
    + *   majority of tables in an instance are published and thus
unfilterable. The
    + *   threshold of 100 was determined to be the optimal balance
based on performance
    + *   tests.
    + *
    + *   Additionally, filtering is paused for transactions
containing WAL records
    + *   (INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION) that modify
the historical
    + *   snapshot constructed during logical decoding. This pause is
necessary because
    + *   constructing a correct historical snapshot for searching publication
    + *   information requires processing these WAL records.
    b.
    + if (!has_tuple)
    + {
    + /*
    + * Mapped catalog tuple without data, emitted while catalog table was in
    + * the process of being rewritten. We can fail to look up the
    + * relfilenumber, because the relmapper has no "historic" view, in
    + * contrast to the normal catalog during decoding. Thus repeated rewrites
    + * can cause a lookup failure. That's OK because we do not decode catalog
    + * changes anyway. Normally such tuples would be skipped over below, but
    + * we can't identify whether the table should be logically logged without
    + * mapping the relfilenumber to the oid.
    + */
    + return NULL;
    + }

2. I think, 'rb->unfiltered_changes_count' should be initialised in
the function 'ReorderBufferAllocate'. While debugging I found that the
initial value of rb->unfiltered_changes_count is 127. I think it
should be set to '0' inside 'ReorderBufferAllocate'. Am I missing
something here?
I have also added the same comment in point 1. in [2]/messages/by-id/CANhcyEUF1HzDRj_fJAGCqBqNg7xGVoATR7XgR311xq8UvBg9tg@mail.gmail.com.

Also, please ignore point 2. in email [2]/messages/by-id/CANhcyEUF1HzDRj_fJAGCqBqNg7xGVoATR7XgR311xq8UvBg9tg@mail.gmail.com a crash happened because I
was testing it without doing a clean build. Sorry for the
inconvenience.

[1]: https://www.postgresql.org/docs/devel/source-format.html
[2]: /messages/by-id/CANhcyEUF1HzDRj_fJAGCqBqNg7xGVoATR7XgR311xq8UvBg9tg@mail.gmail.com

Thanks and Regards,
Shlok Kyal

#39Shlok Kyal
shlok.kyal.oss@gmail.com
In reply to: Ajin Cherian (#37)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Thu, 20 Mar 2025 at 18:09, Ajin Cherian <itsajin@gmail.com> wrote:

On Thu, Mar 13, 2025 at 5:49 PM Ajin Cherian <itsajin@gmail.com> wrote:

Moving this patch to the next CF as this patch needs more design level
inputs which may not be feasible in this CF but do continue to review
the patch.

regards,
Ajin Cherian
Fujitsu Australia

Rebased the patch as it no longer applied.

Hi Ajin,

I reviewed v16-0002 patch, here are my comments:

1. Guc CHANGES_THRESHOLD_FOR_FILTER was introduced in 0001 patch,
should this change be in 0001 patch?

- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing
CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
2. Following comment is repeated inside DecodeInsert, DecodeUpdate,
DecodeDelete, DecodeMultiInsert, DecodeSpecConfirm
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,

Since this comment is the same, should we remove it from the above
functions and add this where function 'SkipThisChange' is defined?

Thanks and Regards,
Shlok Kyal

#40Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#37)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin.

Some review comments for patch v16-0001.

======
Commit message

1.
A hash cache of relation file locators is implemented to cache whether a
relation is filterable or not. This ensures that we only need to retrieve

~

/hash cache/hash table/

(AFAICT you called this a hash table elsewhere in all code comments)

~~~

2.
Additionally, filtering is paused for transactions containing WAL records
(INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION) that modify the historical
snapshot constructed during logical decoding. This pause is necessary because
constructing a correct historical snapshot for searching publication
information requires processing these WAL records.

~

IIUC, the "pause" here is really referring to the 100 changes
following an unfilterable txn. So, I don't think what you are
describing here is a "pause" -- it is just another reason for the tx
to be marked unfilterable, and the pause logic will take care of
itself. So, maybe it should be worded more like

SUGGESTION
Additionally, transactions containing WAL records (INTERNAL_SNAPSHOT,
COMMAND_ID, or INVALIDATION) that modify the historical
snapshot constructed during logical decoding are deemed unfilterable.
This is necessary because constructing a correct historical snapshot
for searching publication information requires processing these WAL
records.

======
src/backend/replication/logical/reorderbuffer.c

3. Header comment.

If you change the commit message based on previous suggestions, then
change the comment here also to match it.

~~~

4.
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100

/every 100/the next 100/

~~~

+ReorderBufferFilterByRelFileLocator:

5.
+ * if we are decoding a transaction without these records. See comment on top
+ * of GetDecodableRelation() to see list of relations that are not
+ * decoded by the reorderbuffer.

/to see list of relations/to see the kinds of relation/

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#41Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#37)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Here is another general review comment for patch 0001.

~~~

I keep getting confused by the distinction between the 2 member fields
that are often found working hand-in-hand:
entry->filterable
rb->can_filter_change

Unfortunately, because the names are very similar I keep blurring the
meanings, and then nothing makes sense.

IIUC, the meanings are actually like:

entry->filterable. This means filtering is *possible* for this kind of
relation; it doesn't mean it will happen though.

rb->can_filter_change. This means the plugin will *try* to filter the
change; it might do nothing if entry->filterable is false;
can_filter_change bool is used for the 100 change "temporary
suspension" logic (e.g. so it if is false we won't even try to filter
despite entry->filterable is true).

If those meanings are accurate I think some better member names might be:
entry->filterable
rb->try_to_filter_change

Also these explanations/distinctions need to be made more clearly in
the commit message and/of file head comments, as well as where those
members are defined.

======
Kind Regards,
Peter Smith.
Fujitsu Australia.

#42Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#37)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Some review comments for patch v16-0002.

======
doc/src/sgml/logicaldecoding.sgml

1.
+      To indicate that decoding can be skipped for the given change
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.

/change <parameter>change_type</parameter>/<parameter>change_type</parameter>/

(don't need to say "change" twice)

======
src/backend/replication/logical/decode.c

SkipThisChange:

2.
+/* Function to determine whether to skip the change */
+static bool SkipThisChange(LogicalDecodingContext *ctx, XLogRecPtr
origptr, TransactionId xid,
+ RelFileLocator *target_locator, ReorderBufferChangeType change_type)
+{
+ return (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, xid, origptr,
target_locator,
+ change_type));
+}

2a.
By convention the function return should be on its own line.

~

2b.
Probably this should be declared *inline* like other functions similar to this?

~

2c.
Consider renaming this function to FilterChange(...). I think that
might align better with the other functions like
FilterChangeIsEnabled, FilterByOrigin etc which all refer to
"filtering" instead of "skipping".

~~~

3.
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+ REORDER_BUFFER_CHANGE_INSERT))

All these block comments prior to the calls to SkipThisChange seem
slightly overkill (I think Shlok also reports this). IMO this comment
can be much simpler:
e.g.
Can the relation associated with this change be skipped?

This is repeated in multiple places: DecodeInsert, DecodeUpdate,
DecodeDelete, DecodeMultiInsert, DecodeSpecConfirm

======
src/backend/replication/logical/logical.c

filter_change_cb_wrapper:

4.
+ /* check if the filter change callback is supported */
+ if (ctx->callbacks.filter_change_cb == NULL)
+ return false;
+

Other optional callbacks are more terse and just say like below, with
no comment.
if (!ctx->callbacks.truncate_cb)
return;
SUGGESTION (do the same here)
if (!ctx->callbacks.filter_change_cb)
return false;

======
src/backend/replication/logical/reorderbuffer.c

5.
 /*
  * After encountering a change that cannot be filtered out, filtering is
- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing
CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
  * This strategy helps to minimize the overhead of performing a hash table
  * search for each record, especially when most changes are not filterable.
  */

I agree with Shlok. This change seems to belong in patch 0001.

~~~

ReorderBufferFilterByRelFileLocator

6.
ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
XLogRecPtr lsn, RelFileLocator *rlocator,
ReorderBufferChangeType change_type)
~

The function comment for this says "Returns true if the relation can
be filtered; otherwise, false.". I'm not sure that it is strictly
correct. IMO the comment should explain more about how the return
boolean depends also on the *change_type*. e.g. IIUC you could be able
to filter INSERTS but not filter DELETES even for the same relation.

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#43Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#37)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Some review comments for patch v16-0003.

======
Patch

1.
mode change 100644 => 100755 src/test/subscription/t/001_rep_changes.pl

What's this mode change for?

======
Commit message

2. missing commit message

======
src/backend/replication/logical/reorderbuffer.c

3.
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0

Hmm. You cannot leave a thing like this in the patch because it seems
like just a temporary hack and means the patch cannot be committed in
this form. It needs some kind of more permanent testing solution,
allowing the tests of this patch can executed efficiently, and the
patch pushed, all without impacting the end-user. Maybe consider if
it's possible to have some injection point so the text can "inject" a
zero here (??).

======
src/test/subscription/t/001_rep_changes.pl

4.
# Create new tables on publisher and subscriber
$node_publisher->safe_psql('postgres', "CREATE TABLE pub_table (id int
primary key, data text)");
$node_publisher->safe_psql('postgres', "CREATE TABLE unpub_table (id
int primary key, data text)");
$node_publisher->safe_psql('postgres', "CREATE TABLE insert_only_table
(id int primary key, data text)");
$node_publisher->safe_psql('postgres', "CREATE TABLE delete_only_table
(id int primary key, data text)");

You could combine all those DDLs.

~~~

5.
$node_subscriber->safe_psql('postgres', "CREATE TABLE pub_table (id
int primary key, data text)");
$node_subscriber->safe_psql('postgres', "CREATE TABLE unpub_table (id
int primary key, data text)");
$node_subscriber->safe_psql('postgres', "CREATE TABLE
insert_only_table (id int primary key, data text)");
$node_subscriber->safe_psql('postgres', "CREATE TABLE
delete_only_table (id int primary key, data text)");

5a.
You could combine all those DDLs.

~

5b.
There are double blank lines here.

~~~

6.
# Setup logical replication publications
$node_publisher->safe_psql('postgres', "CREATE PUBLICATION pub_all FOR
TABLE pub_table");
$node_publisher->safe_psql('postgres', "CREATE PUBLICATION
pub_insert_only FOR TABLE insert_only_table WITH (publish = insert)");
$node_publisher->safe_psql('postgres', "CREATE PUBLICATION
pub_delete_only FOR TABLE delete_only_table WITH (publish = delete)");

You could combine all those DDLs.

~~~

7.
# Setup logical replication subscription
$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION sub_all
CONNECTION '$publisher_connstr' PUBLICATION pub_all");
$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION
sub_insert_only CONNECTION '$publisher_connstr' PUBLICATION
pub_insert_only");
$node_subscriber->safe_psql('postgres', "CREATE SUBSCRIPTION
sub_delete_only CONNECTION '$publisher_connstr' PUBLICATION
pub_delete_only");

7a.
comment should be plural /subscription/subscriptions/

~

7b.
You could combine all those DDL.

~~~

8.
# Insert, delete, and update tests for restricted publication tables
$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "INSERT INTO insert_only_table
VALUES (1, 'to be inserted')");
$node_publisher->safe_psql('postgres', "UPDATE insert_only_table SET
data = 'updated' WHERE id = 1");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering UPDATE/,
'unpublished UPDATE is filtered');

$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "DELETE FROM insert_only_table
WHERE id = 1");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering DELETE/,
'unpublished DELETE is filtered');

$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "INSERT INTO delete_only_table
VALUES (1, 'to be deleted')");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering INSERT/,
'unpublished INSERT is filtered');

~
8a.
Maybe it is clearer to say "...for publications with restricted pubactions"

~

8b.
Change the comment or rearrange the tests so they are in the same
order as described

~

8c.
Looking at the expected logs I wondered if it might be nicer for the
pgoutput_filter_change doing this logging to also emit the relation
name.

~~~

9.
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_all");
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_insert_only");
+$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION sub_delete_only");
+

9a.
You could combine all those DDL

~

9b.
Add some simple comment like "# cleanup"

~

9c.
Any reason why you dropped the subscriptions but not the publications?

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#44Ajin Cherian
itsajin@gmail.com
In reply to: Peter Smith (#43)
3 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Thu, Apr 3, 2025 at 11:01 PM Shlok Kyal <shlok.kyal.oss@gmail.com> wrote:

Hi Ajin,

I have reviewed patch v16-0001, here are my comments:

1. There are some places where comments are more than 80 columns
window. I think it should be <=80 as per [1].

Fixed.

2. I think, 'rb->unfiltered_changes_count' should be initialised in
the function 'ReorderBufferAllocate'. While debugging I found that the
initial value of rb->unfiltered_changes_count is 127. I think it
should be set to '0' inside 'ReorderBufferAllocate'. Am I missing
something here?
I have also added the same comment in point 1. in [2].

Fixed.

On Sat, Apr 5, 2025 at 12:34 AM Shlok Kyal <shlok.kyal.oss@gmail.com> wrote:

On Thu, 20 Mar 2025 at 18:09, Ajin Cherian <itsajin@gmail.com> wrote:

On Thu, Mar 13, 2025 at 5:49 PM Ajin Cherian <itsajin@gmail.com> wrote:

Moving this patch to the next CF as this patch needs more design level
inputs which may not be feasible in this CF but do continue to review
the patch.

regards,
Ajin Cherian
Fujitsu Australia

Rebased the patch as it no longer applied.

Hi Ajin,

I reviewed v16-0002 patch, here are my comments:

1. Guc CHANGES_THRESHOLD_FOR_FILTER was introduced in 0001 patch,
should this change be in 0001 patch?

Fixed.

- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing
CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
2. Following comment is repeated inside DecodeInsert, DecodeUpdate,
DecodeDelete, DecodeMultiInsert, DecodeSpecConfirm
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,

Since this comment is the same, should we remove it from the above
functions and add this where function 'SkipThisChange' is defined?

reworded this.

On Thu, Apr 10, 2025 at 2:26 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin.

Some review comments for patch v16-0001.

======
Commit message

1.
A hash cache of relation file locators is implemented to cache whether a
relation is filterable or not. This ensures that we only need to retrieve

~

/hash cache/hash table/

(AFAICT you called this a hash table elsewhere in all code comments)

Fixed.

~~~

2.
Additionally, filtering is paused for transactions containing WAL records
(INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION) that modify the historical
snapshot constructed during logical decoding. This pause is necessary because
constructing a correct historical snapshot for searching publication
information requires processing these WAL records.

~

IIUC, the "pause" here is really referring to the 100 changes
following an unfilterable txn. So, I don't think what you are
describing here is a "pause" -- it is just another reason for the tx
to be marked unfilterable, and the pause logic will take care of
itself. So, maybe it should be worded more like

SUGGESTION
Additionally, transactions containing WAL records (INTERNAL_SNAPSHOT,
COMMAND_ID, or INVALIDATION) that modify the historical
snapshot constructed during logical decoding are deemed unfilterable.
This is necessary because constructing a correct historical snapshot
for searching publication information requires processing these WAL
records.

Fixed.

======
src/backend/replication/logical/reorderbuffer.c

3. Header comment.

If you change the commit message based on previous suggestions, then
change the comment here also to match it.

~~~

4.
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * This strategy helps to minimize the overhead of performing a hash table
+ * search for each record, especially when most changes are not filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100

/every 100/the next 100/

~~~

+ReorderBufferFilterByRelFileLocator:

5.
+ * if we are decoding a transaction without these records. See comment on top
+ * of GetDecodableRelation() to see list of relations that are not
+ * decoded by the reorderbuffer.

/to see list of relations/to see the kinds of relation/

Fixed.

On Fri, Apr 11, 2025 at 1:15 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Here is another general review comment for patch 0001.

~~~

I keep getting confused by the distinction between the 2 member fields
that are often found working hand-in-hand:
entry->filterable
rb->can_filter_change

Unfortunately, because the names are very similar I keep blurring the
meanings, and then nothing makes sense.

IIUC, the meanings are actually like:

entry->filterable. This means filtering is *possible* for this kind of
relation; it doesn't mean it will happen though.

rb->can_filter_change. This means the plugin will *try* to filter the
change; it might do nothing if entry->filterable is false;
can_filter_change bool is used for the 100 change "temporary
suspension" logic (e.g. so it if is false we won't even try to filter
despite entry->filterable is true).

If those meanings are accurate I think some better member names might be:
entry->filterable
rb->try_to_filter_change

Also these explanations/distinctions need to be made more clearly in
the commit message and/of file head comments, as well as where those
members are defined.

Fixed as recommended.

On Fri, Apr 11, 2025 at 2:21 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v16-0002.

======
doc/src/sgml/logicaldecoding.sgml

1.
+      To indicate that decoding can be skipped for the given change
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.

/change <parameter>change_type</parameter>/<parameter>change_type</parameter>/

(don't need to say "change" twice)

Fixed.

======
src/backend/replication/logical/decode.c

SkipThisChange:

2.
+/* Function to determine whether to skip the change */
+static bool SkipThisChange(LogicalDecodingContext *ctx, XLogRecPtr
origptr, TransactionId xid,
+ RelFileLocator *target_locator, ReorderBufferChangeType change_type)
+{
+ return (FilterChangeIsEnabled(ctx) &&
+ ReorderBufferFilterByRelFileLocator(ctx->reorder, xid, origptr,
target_locator,
+ change_type));
+}

2a.
By convention the function return should be on its own line.

~

2b.
Probably this should be declared *inline* like other functions similar to this?

Fixed.

~

2c.
Consider renaming this function to FilterChange(...). I think that
might align better with the other functions like
FilterChangeIsEnabled, FilterByOrigin etc which all refer to
"filtering" instead of "skipping".

Fixed.

~~~

3.
+ /*
+ * When filtering changes, determine if the relation associated with the change
+ * can be skipped. This could be because the relation is unlogged or because
+ * the plugin has opted to exclude this relation from decoding.
+ */
+ if (SkipThisChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+ REORDER_BUFFER_CHANGE_INSERT))

All these block comments prior to the calls to SkipThisChange seem
slightly overkill (I think Shlok also reports this). IMO this comment
can be much simpler:
e.g.
Can the relation associated with this change be skipped?

This is repeated in multiple places: DecodeInsert, DecodeUpdate,
DecodeDelete, DecodeMultiInsert, DecodeSpecConfirm

Fixed.

======
src/backend/replication/logical/logical.c

filter_change_cb_wrapper:

4.
+ /* check if the filter change callback is supported */
+ if (ctx->callbacks.filter_change_cb == NULL)
+ return false;
+

Other optional callbacks are more terse and just say like below, with
no comment.
if (!ctx->callbacks.truncate_cb)
return;
SUGGESTION (do the same here)
if (!ctx->callbacks.filter_change_cb)
return false;

======
src/backend/replication/logical/reorderbuffer.c

5.
/*
* After encountering a change that cannot be filtered out, filtering is
- * temporarily suspended. Filtering resumes after processing every 100 changes.
+ * temporarily suspended. Filtering resumes after processing
CHANGES_THRESHOLD_FOR_FILTER
+ * changes.
* This strategy helps to minimize the overhead of performing a hash table
* search for each record, especially when most changes are not filterable.
*/

I agree with Shlok. This change seems to belong in patch 0001.

Fixed.

~~~

ReorderBufferFilterByRelFileLocator

6.
ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
XLogRecPtr lsn, RelFileLocator *rlocator,
ReorderBufferChangeType change_type)
~

The function comment for this says "Returns true if the relation can
be filtered; otherwise, false.". I'm not sure that it is strictly
correct. IMO the comment should explain more about how the return
boolean depends also on the *change_type*. e.g. IIUC you could be able
to filter INSERTS but not filter DELETES even for the same relation.

Reworded.

On Fri, Apr 11, 2025 at 4:06 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v16-0003.

======
Patch

1.
mode change 100644 => 100755 src/test/subscription/t/001_rep_changes.pl

What's this mode change for?

Fixed.

======
Commit message

2. missing commit message

======
src/backend/replication/logical/reorderbuffer.c

Fixed.

3.
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0

Hmm. You cannot leave a thing like this in the patch because it seems
like just a temporary hack and means the patch cannot be committed in
this form. It needs some kind of more permanent testing solution,
allowing the tests of this patch can executed efficiently, and the
patch pushed, all without impacting the end-user. Maybe consider if
it's possible to have some injection point so the text can "inject" a
zero here (??).

Yes, this is currently just a hack before we decide whether throttling
buffer value is correct and whether it should be user configurable.

======
src/test/subscription/t/001_rep_changes.pl

4.
# Create new tables on publisher and subscriber
$node_publisher->safe_psql('postgres', "CREATE TABLE pub_table (id int
primary key, data text)");
$node_publisher->safe_psql('postgres', "CREATE TABLE unpub_table (id
int primary key, data text)");
$node_publisher->safe_psql('postgres', "CREATE TABLE insert_only_table
(id int primary key, data text)");
$node_publisher->safe_psql('postgres', "CREATE TABLE delete_only_table
(id int primary key, data text)");

You could combine all those DDLs.

Fixed all these.

Attached patch v17

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

v17-0002-Introduce-an-output-plugin-callback-to-filter-ch.patchapplication/octet-stream; name=v17-0002-Introduce-an-output-plugin-callback-to-filter-ch.patchDownload
From 01a03a25e6bd17ff3ce70c1f20ffedf05c8108b9 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Wed, 16 Apr 2025 08:35:21 -0400
Subject: [PATCH v17 2/3] Introduce an output plugin callback to filter changes

This new output plugin callback provides an option to logical decoding plugins to filter out
changes early. The primary purpose of the callback is to conserve memory and processing cycles
by excluding changes that are not required by output plugins.
---
 doc/src/sgml/logicaldecoding.sgml               | 41 ++++++++++++++-
 src/backend/replication/logical/decode.c        | 47 +++++++++++++++--
 src/backend/replication/logical/logical.c       | 41 +++++++++++++++
 src/backend/replication/logical/reorderbuffer.c | 63 ++++++++++++++++------
 src/backend/replication/pgoutput/pgoutput.c     | 70 +++++++++++++++++++++++++
 src/include/replication/output_plugin.h         | 20 +++++++
 src/include/replication/reorderbuffer.h         | 10 +++-
 src/test/subscription/t/013_partition.pl        |  7 +++
 8 files changed, 278 insertions(+), 21 deletions(-)

diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 1c4ae38..5a5fb21 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -560,6 +560,7 @@ typedef struct OutputPluginCallbacks
     LogicalDecodeCommitCB commit_cb;
     LogicalDecodeMessageCB message_cb;
     LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+    LogicalDecodeFilterChangeCB filter_change_cb;
     LogicalDecodeShutdownCB shutdown_cb;
     LogicalDecodeFilterPrepareCB filter_prepare_cb;
     LogicalDecodeBeginPrepareCB begin_prepare_cb;
@@ -582,8 +583,8 @@ typedef void (*LogicalOutputPluginInit) (struct OutputPluginCallbacks *cb);
      and <function>commit_cb</function> callbacks are required,
      while <function>startup_cb</function>, <function>truncate_cb</function>,
      <function>message_cb</function>, <function>filter_by_origin_cb</function>,
-     and <function>shutdown_cb</function> are optional.
-     If <function>truncate_cb</function> is not set but a
+     <function>shutdown_cb</function>, and <function>filter_change_cb</function>
+     are optional. If <function>truncate_cb</function> is not set but a
      <command>TRUNCATE</command> is to be decoded, the action will be ignored.
     </para>
 
@@ -871,6 +872,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
      </para>
      </sect3>
 
+     <sect3 id="logicaldecoding-output-plugin-filter-change">
+     <title>Change Filter Callback</title>
+
+     <para>
+      The optional <function>filter_change_cb</function> is called before a
+      change record is decoded to determine whether the change can be filtered
+      out.
+<programlisting>
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+                                             Oid relid,
+                                             ReorderBufferChangeType change_type,
+                                             bool in_txn, bool *cache_valid);
+</programlisting>
+      To indicate that decoding can be skipped for the given
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.
+      The <parameter>in_txn</parameter> parameter indicates whether the
+      callback is invoked within a transaction block.
+      When <parameter>in_txn</parameter> is false, and if making a decision to filter a change requires being inside a
+      transaction block, such as needing access to the catalog, set
+      <parameter>*cache_valid</parameter> to <literal>false</literal>.
+      This ensures that the callback will be reinvoked once a transaction block
+      starts. If a decision can be made immediately, set
+      <parameter>*cache_valid</parameter> to <literal>true</literal>.
+     </para>
+     <para>
+      The primary purpose of this callback function is to optimize memory usage
+      and processing efficiency by filtering out changes that are unnecessary for
+      output plugins. It enables output plugins to selectively process relevant
+      changes. Caching filtering decisions locally is recommended, as it enables
+      the callback to provide cached results without repeatedly initiating
+      transactions or querying the catalog. This approach minimizes overhead
+      and improves efficiency during the decoding process.
+     </para>
+     </sect3>
+
     <sect3 id="logicaldecoding-output-plugin-message">
      <title>Generic Message Callback</title>
 
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index fb87772..90c7cc6 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -576,6 +576,17 @@ FilterByOrigin(LogicalDecodingContext *ctx, RepOriginId origin_id)
 }
 
 /*
+ * Check if filtering changes before decoding is supported and we're not suppressing filter
+ * changes currently.
+ */
+static inline bool
+FilterChangeIsEnabled(LogicalDecodingContext *ctx)
+{
+	return (ctx->callbacks.filter_change_cb != NULL &&
+				ctx->reorder->try_to_filter_change);
+}
+
+/*
  * Handle rmgr LOGICALMSG_ID records for LogicalDecodingProcessRecord().
  */
 void
@@ -881,6 +892,16 @@ DecodeAbort(LogicalDecodingContext *ctx, XLogRecordBuffer *buf,
 	UpdateDecodingStats(ctx);
 }
 
+/* Function to determine whether to filter the change */
+static inline bool
+FilterChange(LogicalDecodingContext *ctx, XLogRecPtr origptr, TransactionId xid,
+							RelFileLocator *target_locator, ReorderBufferChangeType change_type)
+{
+	return (FilterChangeIsEnabled(ctx) &&
+			ReorderBufferFilterByRelFileLocator(ctx->reorder, xid, origptr, target_locator,
+												change_type));
+}
+
 /*
  * Parse XLOG_HEAP_INSERT (not MULTI_INSERT!) records into tuplebufs.
  *
@@ -915,9 +936,9 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
-	if (ctx->reorder->try_to_filter_change &&
-			ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
-												buf->origptr, &target_locator))
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+						REORDER_BUFFER_CHANGE_INSERT))
 		return;
 
 	change = ReorderBufferAllocChange(ctx->reorder);
@@ -970,6 +991,11 @@ DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+					   REORDER_BUFFER_CHANGE_UPDATE))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_UPDATE;
 	change->origin_id = XLogRecGetOrigin(r);
@@ -1036,6 +1062,11 @@ DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+					   REORDER_BUFFER_CHANGE_DELETE))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 
 	if (xlrec->flags & XLH_DELETE_IS_SUPER)
@@ -1138,6 +1169,11 @@ DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &rlocator,
+					   REORDER_BUFFER_CHANGE_INSERT))
+		return;
+
 	/*
 	 * We know that this multi_insert isn't for a catalog, so the block should
 	 * always have data even if a full-page write of it is taken.
@@ -1229,6 +1265,11 @@ DecodeSpecConfirm(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+						REORDER_BUFFER_CHANGE_INSERT))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM;
 	change->origin_id = XLogRecGetOrigin(r);
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index a8d2e02..09ee1df 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -74,6 +74,9 @@ static void truncate_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 							   XLogRecPtr message_lsn, bool transactional,
 							   const char *prefix, Size message_size, const char *message);
+static bool filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+								  ReorderBufferChangeType change_type, bool in_txn,
+								  bool *cache_valid);
 
 /* streaming callbacks */
 static void stream_start_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
@@ -220,6 +223,7 @@ StartupDecodingContext(List *output_plugin_options,
 	/* wrap output plugin callbacks, so we can add error context information */
 	ctx->reorder->begin = begin_cb_wrapper;
 	ctx->reorder->apply_change = change_cb_wrapper;
+	ctx->reorder->filter_change = filter_change_cb_wrapper;
 	ctx->reorder->apply_truncate = truncate_cb_wrapper;
 	ctx->reorder->commit = commit_cb_wrapper;
 	ctx->reorder->message = message_cb_wrapper;
@@ -1224,6 +1228,43 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+static bool
+filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+					  ReorderBufferChangeType change_type, bool in_txn,
+					  bool *cache_valid)
+{
+	LogicalDecodingContext *ctx = cache->private_data;
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	if (ctx->callbacks.filter_change_cb == NULL)
+		return false;
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_change";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_change_cb(ctx, relid, change_type, in_txn, cache_valid);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 3594cae..bdf63c9 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -5668,9 +5668,9 @@ RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid)
 			entry->relid == relid)  /* individual flushed relation */
 		{
 			if (hash_search(RelFileLocatorFilterCache,
-							&entry->key,
-							HASH_REMOVE,
-							NULL) == NULL)
+				&entry->key,
+				HASH_REMOVE,
+				NULL) == NULL)
 				elog(ERROR, "hash table corrupted");
 		}
 	}
@@ -5703,17 +5703,19 @@ ReorderBufferMemoryResetcallback(void *arg)
  * management, the results are cached in the 'RelFileLocatorFilterCache' hash
  * table.
  *
- * Returns true if the relation can be filtered; otherwise, false.
+ * Returns true if this change_type can be filtered; otherwise, false.
  */
 bool
 ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
-									 XLogRecPtr lsn, RelFileLocator *rlocator)
+									XLogRecPtr lsn, RelFileLocator *rlocator,
+									ReorderBufferChangeType change_type)
 
 {
-	bool		found;
-	Relation	relation;
-	bool		using_subtxn;
-	Snapshot	snapshot_now;
+	bool        found;
+	bool		cache_valid;
+	Relation    relation;
+	bool        using_subtxn;
+	Snapshot    snapshot_now;
 	ReorderBufferTXN *txn,
 					 *toptxn;
 	ReorderBufferRelFileLocatorEnt *entry;
@@ -5724,7 +5726,6 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
 	Assert(txn);
 	toptxn = rbtxn_get_toptxn(txn);
-	rb->try_to_filter_change = false;
 
 	/*
 	 * We cannot construct an accurate historical snapshot until all pending
@@ -5740,8 +5741,27 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 
 	if (found)
 	{
-	rb->try_to_filter_change = entry->filterable;
-		return entry->filterable;
+		rb->try_to_filter_change = entry->filterable;
+
+		/*
+		 * Quick return if we already know that the relation is not to be
+		 * decoded. These are for special relations that are unlogged and for
+		 * sequences and catalogs.
+		 */
+		if (entry->filterable)
+			return true;
+
+		/* Allow the output plugin to filter relations */
+		rb->try_to_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  false, &cache_valid);
+
+		/*
+		 * If plugin had the relation ready in cache, the response is valid,
+		 * else we'll need to call the plugin a second time within a
+		 * transaction.
+		 */
+		if (cache_valid)
+			return rb->try_to_filter_change;
 	}
 
 	/* constructs a temporary historical snapshot */
@@ -5765,14 +5785,27 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 
 	if (RelationIsValid(relation))
 	{
-		entry->relid = RelationGetRelid(relation);
+		if (IsToastRelation(relation))
+		{
+			char   *toast_name = RelationGetRelationName(relation);
+			int     n PG_USED_FOR_ASSERTS_ONLY;
+
+			n = sscanf(toast_name, "pg_toast_%u", &entry->relid);
+
+			Assert(n == 1);
+		}
+		else
+			entry->relid = RelationGetRelid(relation);
+
 		entry->filterable = false;
+		rb->try_to_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  true, &cache_valid);
 		RelationClose(relation);
 	}
 	else
 	{
 		entry->relid = InvalidOid;
-		entry->filterable = true;
+		rb->try_to_filter_change = entry->filterable = true;
 	}
 
 	rb->try_to_filter_change = entry->filterable;
@@ -5792,7 +5825,7 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (using_subtxn)
 		RollbackAndReleaseCurrentSubTransaction();
 
-	return entry->filterable;
+	return rb->try_to_filter_change;
 }
 
 /*
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index 693a766..c356423 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -60,6 +60,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+								   ReorderBufferChangeType change_type, bool in_txn,
+								   bool *cache_valid);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -271,6 +274,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_change_cb = pgoutput_filter_change;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1749,6 +1753,72 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 }
 
 /*
+ * Determine whether a change to the specified relation should be published.
+ *
+ * See the comments atop of LogicalDecodeFilterChangeCB for details.
+ */
+static bool
+pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+					   ReorderBufferChangeType change_type, bool in_txn, bool *cache_valid)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *entry;
+
+	Assert(RelationSyncCache != NULL);
+
+	if (in_txn)
+	{
+		Relation	relation;
+
+		relation = RelationIdGetRelation(relid);
+		entry = get_rel_sync_entry(data, relation);
+		*cache_valid = true;
+	}
+	else
+	{
+		entry = (RelationSyncEntry *) hash_search(RelationSyncCache,
+												  &relid,
+												  HASH_FIND, cache_valid);
+		if (!*cache_valid)
+			return false;
+	}
+
+	/*
+	 * If the pubaction is not supported by this publication then return true to say
+	 * the change for this entry can be skipped.
+	 */
+	switch (change_type)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			if (!entry->pubactions.pubinsert)
+			{
+				elog(DEBUG1, "Filtering INSERT");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			if (!entry->pubactions.pubupdate)
+			{
+				elog(DEBUG1, "Filtering UPDATE");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			if (!entry->pubactions.pubdelete)
+			{
+				elog(DEBUG1, "Filtering DELETE");
+				return true;
+			}
+			break;
+		default:
+			/* allow any other changes that are not explicitly filtered */
+			return false;
+	}
+
+	return false;
+}
+
+/*
  * Shutdown the output plugin.
  *
  * Note, we don't need to clean the data->context, data->cachectx, and
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 8d4d5b71..f82bc1d 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -97,6 +97,25 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
 											   RepOriginId origin_id);
 
 /*
+ * This callback is called before a change record is decoded to determine
+ * whether the change can be filtered out before entering the reorder buffer.
+ *
+ * Typically, the output plugin needs to refer to the system catalog to make
+ * this decision. To enhance efficiency, the plugin should cache the lookup
+ * result for each relation, minimizing catalog access on subsequent calls.
+ *
+ * If the callback is called with 'in_txn' set as false (the reorder
+ * buffer has not started a transaction), the output plugin should set '*cache_valid'
+ * to false to indicate that the result is not available in its internal cache.
+ * If 'in_txn' is true, the plugin can create a cache entry after querying the
+ * catalog.
+ */
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
+/*
  * Called to shutdown an output plugin.
  */
 typedef void (*LogicalDecodeShutdownCB) (struct LogicalDecodingContext *ctx);
@@ -222,6 +241,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterChangeCB filter_change_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 0727856..4202ac7 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -494,6 +494,12 @@ typedef void (*ReorderBufferMessageCB) (ReorderBuffer *rb,
 										const char *prefix, Size sz,
 										const char *message);
 
+/* filter change callback signature */
+typedef bool (*ReorderBufferFilterChangeCB) (ReorderBuffer *rb,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
 /* begin prepare callback signature */
 typedef void (*ReorderBufferBeginPrepareCB) (ReorderBuffer *rb,
 											 ReorderBufferTXN *txn);
@@ -604,6 +610,7 @@ struct ReorderBuffer
 	 */
 	ReorderBufferBeginCB begin;
 	ReorderBufferApplyChangeCB apply_change;
+	ReorderBufferFilterChangeCB filter_change;
 	ReorderBufferApplyTruncateCB apply_truncate;
 	ReorderBufferCommitCB commit;
 	ReorderBufferMessageCB message;
@@ -779,7 +786,8 @@ extern uint32 ReorderBufferGetInvalidations(ReorderBuffer *rb,
 extern void StartupReorderBuffer(void);
 
 extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
-												XLogRecPtr lsn, RelFileLocator *rlocator);
+												XLogRecPtr lsn, RelFileLocator *rlocator,
+												ReorderBufferChangeType change_type);
 extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
 
 #endif
diff --git a/src/test/subscription/t/013_partition.pl b/src/test/subscription/t/013_partition.pl
index 61b0cb4..a64b27e 100644
--- a/src/test/subscription/t/013_partition.pl
+++ b/src/test/subscription/t/013_partition.pl
@@ -472,6 +472,13 @@ $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4 (a int PRIMARY KEY)");
 $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4_1 (a int PRIMARY KEY)");
+
+# Ensure that the subscription 'sub2' catches up with the latest changes. This
+# is necessary for the walsender to update its historic snapshot. Otherwise,
+# the walsender might retain an outdated snapshot, potentially preventing it
+# from accessing the newly created publication.
+$node_publisher->wait_for_catchup('sub2');
+
 # Since we specified publish_via_partition_root in pub_all and
 # pub_lower_level, all partition tables use their root tables' identity and
 # schema. We set the list of publications so that the FOR ALL TABLES
-- 
1.8.3.1

v17-0003-Tests-for-filtering-unpublished-changes.patchapplication/octet-stream; name=v17-0003-Tests-for-filtering-unpublished-changes.patchDownload
From 719073e0a26fff7cac5ce83896958987528ad7be Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Wed, 16 Apr 2025 23:36:00 -0400
Subject: [PATCH v17 3/3] Tests for filtering unpublished changes

---
 src/backend/replication/logical/reorderbuffer.c |  2 +-
 src/test/subscription/t/001_rep_changes.pl      | 75 +++++++++++++++++++++++++
 2 files changed, 76 insertions(+), 1 deletion(-)

diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index bdf63c9..98dca48 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -378,7 +378,7 @@ static void ReorderBufferMemoryResetcallback(void *arg);
  * hash table search for each record, especially when most changes are not
  * filterable.
  */
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0
 
 /*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
diff --git a/src/test/subscription/t/001_rep_changes.pl b/src/test/subscription/t/001_rep_changes.pl
index 916fdb4..cfdc642 100644
--- a/src/test/subscription/t/001_rep_changes.pl
+++ b/src/test/subscription/t/001_rep_changes.pl
@@ -571,6 +571,81 @@ $result = $node_subscriber->safe_psql('postgres',
 	"SELECT count(*) FROM pg_replication_origin");
 is($result, qq(0), 'check replication origin was dropped on subscriber');
 
+$node_publisher->stop('fast');
+
+# Additional tests for filtering of unpublished changes
+# Bump up log verbosity to DEBUG1 for confirmation logs
+$node_publisher->append_conf('postgresql.conf', "log_min_messages = debug1");
+$node_publisher->start;
+
+# Create new tables on publisher and subscriber
+$node_publisher->safe_psql('postgres',
+							"CREATE TABLE pub_table (id int primary key, data text);
+							 CREATE TABLE unpub_table (id int primary key, data text);
+							 CREATE TABLE insert_only_table (id int primary key, data text);
+							 CREATE TABLE delete_only_table (id int primary key, data text);");
+
+$node_subscriber->safe_psql('postgres',
+							"CREATE TABLE pub_table (id int primary key, data text);
+							 CREATE TABLE unpub_table (id int primary key, data text);
+							 CREATE TABLE insert_only_table (id int primary key, data text);
+							 CREATE TABLE delete_only_table (id int primary key, data text);");
+
+# Setup logical replication publications
+$node_publisher->safe_psql('postgres',
+	"CREATE PUBLICATION pub_all FOR TABLE pub_table;
+	 CREATE PUBLICATION pub_insert_only FOR TABLE insert_only_table WITH (publish = insert);
+	 CREATE PUBLICATION pub_delete_only FOR TABLE delete_only_table WITH (publish = delete);");
+
+# Setup logical replication subscriptions
+$node_subscriber->safe_psql('postgres',
+"CREATE SUBSCRIPTION sub_all CONNECTION '$publisher_connstr' PUBLICATION pub_all;
+ CREATE SUBSCRIPTION sub_insert_only CONNECTION '$publisher_connstr' PUBLICATION pub_insert_only;
+ CREATE SUBSCRIPTION sub_delete_only CONNECTION '$publisher_connstr' PUBLICATION pub_delete_only;");
+
+# Wait for initial sync
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_all');
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_insert_only');
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_delete_only');
+
+# Insert into an unpublished table (should not be replicated)
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO unpub_table VALUES (1, 'unpublished')");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+
+# Insert, update, and delete tests for publication with restricted tables
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO insert_only_table VALUES (1, 'to be inserted')");
+$node_publisher->safe_psql('postgres', "UPDATE insert_only_table SET data = 'updated' WHERE id = 1");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering UPDATE/,
+	'unpublished UPDATE is filtered');
+
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "DELETE FROM insert_only_table WHERE id = 1");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering DELETE/,
+	'unpublished DELETE is filtered');
+
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO delete_only_table VALUES (1, 'to be deleted')");
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+
+#cleanup
+$node_subscriber->safe_psql('postgres',
+					"DROP SUBSCRIPTION sub_all;
+					 DROP SUBSCRIPTION sub_insert_only;
+					 DROP SUBSCRIPTION sub_delete_only;");
+
+$node_publisher->safe_psql('postgres',
+					"DROP PUBLICATION pub_all;
+					 DROP PUBLICATION pub_insert_only;
+					 DROP PUBLICATION pub_delete_only;");
+
 $node_subscriber->stop('fast');
 $node_publisher->stop('fast');
 
-- 
1.8.3.1

v17-0001-Filter-transactions-that-need-not-be-published.patchapplication/octet-stream; name=v17-0001-Filter-transactions-that-need-not-be-published.patchDownload
From f027bf82dc203f0ad179ed6450f64c6f865350db Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Wed, 16 Apr 2025 08:29:41 -0400
Subject: [PATCH v17 1/3] Filter transactions that need not be published.

This patch set aims to filter transactions at the decode stage rather than
streaming time, which allows the system to skip processing transactions that do
not contain tables included in the logical replication walsender's publication
list. As a result, this can prevent the serialization of unnecessary records to
disk during non-streaming mode, especially in large transactions, and also
reduce memory and CPU usage in streaming mode when many changes can be filtered
out.

A hash table of relation file locators is implemented to cache whether a
relation is filterable or not. This ensures that we only need to retrieve
relation and publication information from the catalog when a cache entry is
invalid, avoiding the overhead of starting a transaction for each record.

Filtering is temporarily suspended for a sequence of changes (set to 100
changes) when an unfilterable change is encountered. This strategy minimizes
the overhead of hash searching for every record, which is beneficial when the
majority of tables in an instance are published and thus unfilterable. The
threshold of 100 was determined to be the optimal balance based on performance
tests.

Additionally, transactions containing WAL records (INTERNAL_SNAPSHOT,
COMMAND_ID, or INVALIDATION) that modify the historical snapshot
constructed during logical decoding are deemed unfilterable. This
is necessary because constructing a correct historical snapshot
for searching publication information requires processing these WAL record.

Note that this patch filters changes only for system catalog relations,
non-logically logged relations, or temporary heaps and sequences. A subsequent
patch will introduce a new output plugin in pgoutput, which will further filter
changes for relations not included in publications.
---
 src/backend/replication/logical/decode.c        |   5 +
 src/backend/replication/logical/reorderbuffer.c | 472 ++++++++++++++++++++----
 src/include/replication/reorderbuffer.h         |  18 +
 src/tools/pgindent/typedefs.list                |   2 +
 4 files changed, 428 insertions(+), 69 deletions(-)

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 78f9a0a..fb87772 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -915,6 +915,11 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	if (ctx->reorder->try_to_filter_change &&
+			ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+												buf->origptr, &target_locator))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	if (!(xlrec->flags & XLH_INSERT_IS_SPECULATIVE))
 		change->action = REORDER_BUFFER_CHANGE_INSERT;
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 6765511..3594cae 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -82,6 +82,33 @@
  *	  a bit more memory to the oldest subtransactions, because it's likely
  *	  they are the source for the next sequence of changes.
  *
+ *	  We also try to filter changes at the decode stage rather than at
+ *	  streaming time, which allows the system to skip processing txns that
+ *	  do not contain tables that need to be decoded. As a result,
+ *	  this can prevent the serialization of unnecessary records to disk
+ *	  during non-streaming mode, especially in large transactions, and also
+ *	  reduce memory and CPU usage in streaming mode when many changes can be
+ *	  filtered out.
+ *
+ *	  A hash table of relation file locators is implemented to cache whether a
+ *	  relation is filterable or not. This ensures that we only need to retrieve
+ *	  relation and publication information from the catalog when a cache entry
+ *	  is invalid, avoiding the overhead of starting a transaction for each
+ *	  record.
+ *
+ *	  Filtering is temporarily suspended for a sequence of changes (set to 100
+ *	  changes) when an unfilterable change is encountered. This strategy
+ *	  minimizes the overhead of hash searching for every record, which is
+ *    beneficial when the majority of tables in an instance are published and
+ *	  thus unfilterable. The threshold of 100 was determined to be the optimal
+ *	  balance based on performance tests.
+ *
+ *	  Additionally, transactions containing WAL records (INTERNAL_SNAPSHOT,
+ *	  COMMAND_ID, or INVALIDATION) that modify the historical snapshot
+ *	  constructed during logical decoding are deemed unfilterable. This is
+ *	  necessary because constructing a correct historical snapshot for
+ *	  searching publication information requires processing these WAL records.
+ *
  * -------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -109,6 +136,7 @@
 #include "storage/procarray.h"
 #include "storage/sinval.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 #include "utils/relfilenumbermap.h"
@@ -227,8 +255,10 @@ static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
 											   XLogRecPtr lsn, bool create_as_top);
 static void ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
 											  ReorderBufferTXN *subtxn);
-
 static void AssertTXNLsnOrder(ReorderBuffer *rb);
+static Relation GetDecodableRelation(ReorderBuffer *rb,
+										 RelFileLocator *rlocator,
+										 bool has_tuple);
 
 /* ---------------------------------------
  * support functions for lsn-order iterating over the ->changes of a
@@ -279,6 +309,8 @@ static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
  */
 static inline bool ReorderBufferCanStream(ReorderBuffer *rb);
 static inline bool ReorderBufferCanStartStreaming(ReorderBuffer *rb);
+static Snapshot ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb,
+											   ReorderBufferTXN *txn);
 static void ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
 static void ReorderBufferStreamCommit(ReorderBuffer *rb, ReorderBufferTXN *txn);
 
@@ -305,6 +337,50 @@ static void ReorderBufferChangeMemoryUpdate(ReorderBuffer *rb,
 											bool addition, Size sz);
 
 /*
+ * ---------------------------------------
+ * RelFileLocator filtering
+ * ---------------------------------------
+ * This hash table serves as a lookup table for determining if a relation can
+ * be filtered before being decoded and queued into the buffer.
+ *
+ * The hash table shares the same lifespan as the reorder buffer. This is
+ * crucial because each reorderbuffer may have different configurations or be
+ * associated with different output plugins, affecting the types of tables to
+ * be processed.
+ */
+
+static HTAB *RelFileLocatorFilterCache = NULL;
+
+static bool relation_callbacks_registered = false;
+
+typedef struct ReorderBufferRelFileLocatorKey
+{
+	Oid			reltablespace;
+	RelFileNumber relfilenumber;
+} ReorderBufferRelFileLocatorKey;
+
+/* Hash table entry used to determine if the relation can be filtered. */
+typedef struct ReorderBufferRelFileLocatorEnt
+{
+	ReorderBufferRelFileLocatorKey key;
+	Oid			relid;
+	bool		filterable; /* if true, belongs to a relation that can be filtered */
+} ReorderBufferRelFileLocatorEnt;
+
+static void RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid);
+static void ReorderBufferMemoryResetcallback(void *arg);
+
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing the next
+ * CHANGES_THRESHOLD_FOR_FILTER changes.
+ * This strategy helps to minimize the overhead of performing a
+ * hash table search for each record, especially when most changes are not
+ * filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100
+
+/*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
  * prior ReorderBuffer instances for the same slot.
  */
@@ -362,6 +438,34 @@ ReorderBufferAllocate(void)
 	buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
 								 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
+	/*
+	 * To support early filtering of changes, this hash table acts
+	 * as a lookup table to determine if the corresponding relation is
+	 * required to be decoded and queued into the buffer. The hash table
+	 * shares the same lifespan as the reorder buffer.
+	 *
+	 * Also setup the callback to invalidate cache when relations are updated.
+	 */
+	hash_ctl.keysize = sizeof(ReorderBufferRelFileLocatorKey);
+	hash_ctl.entrysize = sizeof(ReorderBufferRelFileLocatorEnt);
+	hash_ctl.hcxt = buffer->context;
+
+	RelFileLocatorFilterCache =
+		hash_create("RelFileLocatorFilterCache", 1000, &hash_ctl,
+					HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	buffer->relfile_callback.arg = NULL;
+	buffer->relfile_callback.func = ReorderBufferMemoryResetcallback;
+	MemoryContextRegisterResetCallback(buffer->context, &buffer->relfile_callback);
+
+	if (!relation_callbacks_registered)
+	{
+		/* Watch for invalidation events. */
+		CacheRegisterRelcacheCallback(RelFileLocatorCacheInvalidateCallback,
+									  (Datum) 0);
+		relation_callbacks_registered = true;
+	}
+
 	buffer->by_txn_last_xid = InvalidTransactionId;
 	buffer->by_txn_last_txn = NULL;
 
@@ -372,6 +476,9 @@ ReorderBufferAllocate(void)
 	/* txn_heap is ordered by transaction size */
 	buffer->txn_heap = pairingheap_allocate(ReorderBufferTXNSizeCompare, NULL);
 
+	buffer->try_to_filter_change = true;
+	buffer->unfiltered_changes_count = 0;
+
 	buffer->spillTxns = 0;
 	buffer->spillCount = 0;
 	buffer->spillBytes = 0;
@@ -826,6 +933,14 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 
 		toptxn->txn_flags |= RBTXN_HAS_STREAMABLE_CHANGE;
 	}
+	else if (change->action == REORDER_BUFFER_CHANGE_INVALIDATION ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID)
+	{
+		ReorderBufferTXN *toptxn = rbtxn_get_toptxn(txn);
+
+		toptxn->txn_flags |= RBTXN_HAS_SNAPSHOT_CHANGES;
+	}
 
 	change->lsn = lsn;
 	change->txn = txn;
@@ -835,6 +950,17 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 	txn->nentries++;
 	txn->nentries_mem++;
 
+	/*
+	 * If filtering is currently suspended and we've crossed the change threshold,
+	 * attempt to filter again
+	 */
+	if (!rb->try_to_filter_change && (++rb->unfiltered_changes_count
+									>= CHANGES_THRESHOLD_FOR_FILTER))
+	{
+		rb->try_to_filter_change = true;
+		rb->unfiltered_changes_count = 0;
+	}
+
 	/* update memory accounting information */
 	ReorderBufferChangeMemoryUpdate(rb, change, NULL, true,
 									ReorderBufferChangeSize(change));
@@ -1734,6 +1860,9 @@ ReorderBufferTruncateTXN(ReorderBuffer *rb, ReorderBufferTXN *txn, bool txn_prep
 		txn->txn_flags |= RBTXN_IS_SERIALIZED_CLEAR;
 	}
 
+	/* All snapshot changes up to this point have been processed. */
+	txn->txn_flags &= ~RBTXN_HAS_SNAPSHOT_CHANGES;
+
 	/* also reset the number of entries in the transaction */
 	txn->nentries_mem = 0;
 	txn->nentries = 0;
@@ -2177,6 +2306,85 @@ ReorderBufferResetTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 }
 
 /*
+ * Return the relation corresponding to the given RelFileLocator 'rlocator' if
+ * relevant to decoding. Otherwise, return NULL.
+ *
+ * We don't decode catalog files, relations that are not logically logged,
+ * temporary heaps and sequences.
+ */
+static Relation
+GetDecodableRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+						 bool has_tuple)
+{
+	bool		filterable = false;
+	Relation	relation;
+	Oid			reloid;
+	RelPathStr  path;
+
+	reloid = RelidByRelfilenumber(rlocator->spcOid, rlocator->relNumber);
+	path = relpathperm(*rlocator, MAIN_FORKNUM);
+
+	if (!OidIsValid(reloid))
+	{
+		if (!has_tuple)
+		{
+			/*
+			 * Mapped catalog tuple without data, emitted while catalog table
+			 * was in the process of being rewritten. We can fail to look up
+			 * the relfilenumber, because the relmapper has no "historic" view,
+			 * in contrast to the normal catalog during decoding. Thus repeated
+			 * rewrites can cause a lookup failure. That's OK because we do not
+			 * decode catalog changes anyway. Normally such tuples would be
+			 * skipped over below, but we can't identify whether the table
+			 * should be logically logged without mapping the relfilenumber
+			 * to the oid.
+			 */
+			return NULL;
+		}
+
+		elog(ERROR, "could not map filenumber \"%s\" to relation OID",
+				path.str);
+	}
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid, path.str);
+
+	if (!RelationIsLogicallyLogged(relation))
+		filterable = true;
+
+	else if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		/*
+		 * Ignore temporary heaps created during DDL unless the plugin has asked
+		 * for them.
+		 */
+		filterable = true;
+	}
+
+	else if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		/*
+		 * For now ignore sequence changes entirely. Most of the time they don't
+		 * log changes using records we understand, so it doesn't make sense to
+		 * handle the few cases we do.
+		 */
+		filterable = true;
+	}
+
+	/* Return NULL to indicate that this relation need not be decoded. */
+	if (filterable)
+	{
+		RelationClose(relation);
+		return NULL;
+	}
+
+	return relation;
+}
+
+/*
  * Helper function for ReorderBufferReplay and ReorderBufferStreamTXN.
  *
  * Send data of a transaction (and its subtransactions) to the
@@ -2248,7 +2456,6 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 		while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
 		{
 			Relation	relation = NULL;
-			Oid			reloid;
 
 			CHECK_FOR_INTERRUPTS();
 
@@ -2307,55 +2514,10 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 				case REORDER_BUFFER_CHANGE_DELETE:
 					Assert(snapshot_now);
 
-					reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
-												  change->data.tp.rlocator.relNumber);
-
-					/*
-					 * Mapped catalog tuple without data, emitted while
-					 * catalog table was in the process of being rewritten. We
-					 * can fail to look up the relfilenumber, because the
-					 * relmapper has no "historic" view, in contrast to the
-					 * normal catalog during decoding. Thus repeated rewrites
-					 * can cause a lookup failure. That's OK because we do not
-					 * decode catalog changes anyway. Normally such tuples
-					 * would be skipped over below, but we can't identify
-					 * whether the table should be logically logged without
-					 * mapping the relfilenumber to the oid.
-					 */
-					if (reloid == InvalidOid &&
-						change->data.tp.newtuple == NULL &&
-						change->data.tp.oldtuple == NULL)
-						goto change_done;
-					else if (reloid == InvalidOid)
-						elog(ERROR, "could not map filenumber \"%s\" to relation OID",
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM).str);
-
-					relation = RelationIdGetRelation(reloid);
-
+					relation = GetDecodableRelation(rb, &change->data.tp.rlocator,
+														(change->data.tp.newtuple != NULL ||
+														 change->data.tp.oldtuple != NULL));
 					if (!RelationIsValid(relation))
-						elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
-							 reloid,
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM).str);
-
-					if (!RelationIsLogicallyLogged(relation))
-						goto change_done;
-
-					/*
-					 * Ignore temporary heaps created during DDL unless the
-					 * plugin has asked for them.
-					 */
-					if (relation->rd_rel->relrewrite && !rb->output_rewrites)
-						goto change_done;
-
-					/*
-					 * For now ignore sequence changes entirely. Most of the
-					 * time they don't log changes using records we
-					 * understand, so it doesn't make sense to handle the few
-					 * cases we do.
-					 */
-					if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
 						goto change_done;
 
 					/* user-triggered change */
@@ -4171,19 +4333,13 @@ ReorderBufferCanStartStreaming(ReorderBuffer *rb)
 }
 
 /*
- * Send data of a large transaction (and its subtransactions) to the
- * output plugin, but using the stream API.
+ * This function generates or retrieves a consistent snapshot of a transaction
+ * that is currently in progress for logical replication.
  */
-static void
-ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+static Snapshot
+ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb, ReorderBufferTXN *txn)
 {
 	Snapshot	snapshot_now;
-	CommandId	command_id;
-	Size		stream_bytes;
-	bool		txn_is_streamed;
-
-	/* We can never reach here for a subtransaction. */
-	Assert(rbtxn_is_toptxn(txn));
 
 	/*
 	 * We can't make any assumptions about base snapshot here, similar to what
@@ -4226,12 +4382,11 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		if (txn->base_snapshot == NULL)
 		{
 			Assert(txn->ninvalidations == 0);
-			return;
+			return NULL;
 		}
 
-		command_id = FirstCommandId;
 		snapshot_now = ReorderBufferCopySnap(rb, txn->base_snapshot,
-											 txn, command_id);
+											 txn, FirstCommandId);
 	}
 	else
 	{
@@ -4244,17 +4399,40 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		 * the LSN condition in the previous branch (so no need to walk
 		 * through subxacts again). In fact, we must not do that as we may be
 		 * using snapshot half-way through the subxact.
-		 */
-		command_id = txn->command_id;
-
-		/*
+		 *
 		 * We can't use txn->snapshot_now directly because after the last
 		 * streaming run, we might have got some new sub-transactions. So we
 		 * need to add them to the snapshot.
 		 */
 		snapshot_now = ReorderBufferCopySnap(rb, txn->snapshot_now,
-											 txn, command_id);
+											 txn, txn->command_id);
+	}
+
+	return snapshot_now;
+}
+
+/*
+ * Send data of a large transaction (and its subtransactions) to the
+ * output plugin, but using the stream API.
+ */
+static void
+ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+	Snapshot	snapshot_now;
+	Size		stream_bytes;
+	bool		txn_is_streamed;
+
+	/* We can never reach here for a subtransaction. */
+	Assert(rbtxn_is_toptxn(txn));
+
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, txn);
+
+	/* This transaction didn't make any changes to the database till now. */
+	if (snapshot_now == NULL)
+		return;
 
+	if (txn->snapshot_now != NULL)
+	{
 		/* Free the previously copied snapshot. */
 		Assert(txn->snapshot_now->copied);
 		ReorderBufferFreeSnap(rb, txn->snapshot_now);
@@ -4272,7 +4450,7 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 
 	/* Process and send the changes to output plugin. */
 	ReorderBufferProcessTXN(rb, txn, InvalidXLogRecPtr, snapshot_now,
-							command_id, true);
+							snapshot_now->curcid, true);
 
 	rb->streamCount += 1;
 	rb->streamBytes += stream_bytes;
@@ -5462,6 +5640,162 @@ restart:
 }
 
 /*
+ * Flush mapping entries when the corresponding relations are updated in pg_class.
+ * If relid is InvalidOid then do a complete reset of the RelFileLocatorFilterCache.
+ */
+static void
+RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid)
+{
+	HASH_SEQ_STATUS status;
+	ReorderBufferRelFileLocatorEnt *entry;
+
+	if (!RelFileLocatorFilterCache)
+		return;
+
+	hash_seq_init(&status, RelFileLocatorFilterCache);
+
+	/* Slightly inefficient algorithm but not performance critical path */
+	while ((entry = (ReorderBufferRelFileLocatorEnt *) hash_seq_search(&status)) != NULL)
+	{
+		/*
+		 * If a complete reset is requested (when relid parameter is InvalidOid),
+		 * we must remove all entries, otherwise just remove the specific relation's
+		 * entry. Remove any invalid cache entries (these are indicated by invalid
+		 * entry->relid)
+		 */
+		if (relid == InvalidOid ||  /* complete reset */
+			entry->relid == InvalidOid ||   /* invalid cache entry */
+			entry->relid == relid)  /* individual flushed relation */
+		{
+			if (hash_search(RelFileLocatorFilterCache,
+							&entry->key,
+							HASH_REMOVE,
+							NULL) == NULL)
+				elog(ERROR, "hash table corrupted");
+		}
+	}
+}
+
+/*
+ * Context reset/delete callback for RelFileLocatorFilterCache.
+ */
+static void
+ReorderBufferMemoryResetcallback(void *arg)
+{
+	RelFileLocatorFilterCache = NULL;
+}
+
+/*
+ * Determine whether the record corresponding to the relation identified by
+ * 'rlocator' needs to be filtered and not decoded and queued in the buffer.
+ *
+ * Determining the necessity of decoding requires accessing relation
+ * information from the system catalog, which requires a historical snapshot.
+ * We cannot directly use the current snapshot of the transaction due to the
+ * presence of INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION records in the
+ * buffer that could modify the snapshot. A proper snapshot can only be
+ * constructed after these records are processed in ReorderBufferProcessTXN, or
+ * if we are decoding a transaction without these records. See comment on top
+ * of GetDecodableRelation() to see the kinds of relations that are not
+ * decoded by the reorderbuffer.
+ *
+ * To reduce the overhead from the system catalog access and transaction
+ * management, the results are cached in the 'RelFileLocatorFilterCache' hash
+ * table.
+ *
+ * Returns true if the relation can be filtered; otherwise, false.
+ */
+bool
+ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+									 XLogRecPtr lsn, RelFileLocator *rlocator)
+
+{
+	bool		found;
+	Relation	relation;
+	bool		using_subtxn;
+	Snapshot	snapshot_now;
+	ReorderBufferTXN *txn,
+					 *toptxn;
+	ReorderBufferRelFileLocatorEnt *entry;
+	ReorderBufferRelFileLocatorKey key;
+
+	Assert(RelFileLocatorFilterCache);
+
+	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+	Assert(txn);
+	toptxn = rbtxn_get_toptxn(txn);
+	rb->try_to_filter_change = false;
+
+	/*
+	 * We cannot construct an accurate historical snapshot until all pending
+	 * records in this transaction that might update the snapshot are
+	 * processed.
+	 */
+	if (rbtxn_has_snapshot_changes(toptxn))
+		return false;
+
+	key.reltablespace = rlocator->spcOid;
+	key.relfilenumber = rlocator->relNumber;
+	entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+	if (found)
+	{
+	rb->try_to_filter_change = entry->filterable;
+		return entry->filterable;
+	}
+
+	/* constructs a temporary historical snapshot */
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, toptxn);
+	Assert(snapshot_now);
+
+	/* build data to be able to lookup the CommandIds of catalog tuples */
+	ReorderBufferBuildTupleCidHash(rb, toptxn);
+
+	/* setup the initial snapshot */
+	SetupHistoricSnapshot(snapshot_now, toptxn->tuplecid_hash);
+
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by RelFileLocator");
+	else
+		StartTransactionCommand();
+
+	relation = GetDecodableRelation(rb, rlocator, true);
+
+	if (RelationIsValid(relation))
+	{
+		entry->relid = RelationGetRelid(relation);
+		entry->filterable = false;
+		RelationClose(relation);
+	}
+	else
+	{
+		entry->relid = InvalidOid;
+		entry->filterable = true;
+	}
+
+	rb->try_to_filter_change = entry->filterable;
+
+	ReorderBufferFreeSnap(rb, snapshot_now);
+
+	TeardownHistoricSnapshot(false);
+
+	/*
+	 * Aborting the current (sub-)transaction as a whole has the right
+	 * semantics. We want all locks acquired in here to be released, not
+	 * reassigned to the parent and we do not want any database access have
+	 * persistent effects.
+	 */
+	AbortCurrentTransaction();
+
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	return entry->filterable;
+}
+
+/*
  * Count invalidation messages of specified transaction.
  *
  * Returns number of messages, and msgs is set to the pointer of the linked
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 24e88c4..0727856 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -176,6 +176,7 @@ typedef struct ReorderBufferChange
 #define RBTXN_SENT_PREPARE			0x0200
 #define RBTXN_IS_COMMITTED			0x0400
 #define RBTXN_IS_ABORTED			0x0800
+#define RBTXN_HAS_SNAPSHOT_CHANGES  0x1000
 
 #define RBTXN_PREPARE_STATUS_MASK	(RBTXN_IS_PREPARED | RBTXN_SKIPPED_PREPARE | RBTXN_SENT_PREPARE)
 
@@ -215,6 +216,12 @@ typedef struct ReorderBufferChange
 	((txn)->txn_flags & RBTXN_HAS_STREAMABLE_CHANGE) != 0 \
 )
 
+/* Does this transaction make changes to the current snapshot? */
+#define rbtxn_has_snapshot_changes(txn) \
+( \
+	((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
+)
+
 /*
  * Has this transaction been streamed to downstream?
  *
@@ -641,6 +648,7 @@ struct ReorderBuffer
 	 * Private memory context.
 	 */
 	MemoryContext context;
+	MemoryContextCallback relfile_callback;
 
 	/*
 	 * Memory contexts for specific types objects
@@ -661,6 +669,12 @@ struct ReorderBuffer
 	/* Max-heap for sizes of all top-level and sub transactions */
 	pairingheap *txn_heap;
 
+	/* should we try to filter the change? */
+	bool		try_to_filter_change;
+
+	/* number of changes after a failed attempt at filtering */
+	int8		unfiltered_changes_count;
+
 	/*
 	 * Statistics about transactions spilled to disk.
 	 *
@@ -764,4 +778,8 @@ extern uint32 ReorderBufferGetInvalidations(ReorderBuffer *rb,
 
 extern void StartupReorderBuffer(void);
 
+extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+												XLogRecPtr lsn, RelFileLocator *rlocator);
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
+
 #endif
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index d16bc20..1ff1a2b 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2508,6 +2508,8 @@ ReorderBufferIterTXNEntry
 ReorderBufferIterTXNState
 ReorderBufferMessageCB
 ReorderBufferPrepareCB
+ReorderBufferRelFileLocatorEnt
+ReorderBufferRelFileLocatorKey
 ReorderBufferRollbackPreparedCB
 ReorderBufferStreamAbortCB
 ReorderBufferStreamChangeCB
-- 
1.8.3.1

#45Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#44)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Some review comments for patch v17-0001

======
Commit message

1.
Additionally, transactions containing WAL records (INTERNAL_SNAPSHOT,
COMMAND_ID, or INVALIDATION) that modify the historical snapshot
constructed during logical decoding are deemed unfilterable. This
is necessary because constructing a correct historical snapshot
for searching publication information requires processing these WAL record.

~

/these WAL record./these WAL records./

======
src/backend/replication/logical/reorderbuffer.c

ReorderBufferFilterByRelFileLocator:

2.
+ if (found)
+ {
+ rb->try_to_filter_change = entry->filterable;
+ return entry->filterable;
+ }
+

Bad indentation.

======
src/include/replication/reorderbuffer.h

3.
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
+

Why is this extern here? This function is not implemented in patch 0001.

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#46Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#44)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin,

Some review comments for patch v17-0002.

======
src/backend/replication/logical/decode.c

1.
 /*
+ * Check if filtering changes before decoding is supported and we're
not suppressing filter
+ * changes currently.
+ */
+static inline bool
+FilterChangeIsEnabled(LogicalDecodingContext *ctx)
+{
+ return (ctx->callbacks.filter_change_cb != NULL &&
+ ctx->reorder->try_to_filter_change);
+}
+

I still have doubts about the need for/benefits of this to be a
separate function.

It is only called from *one* place within the other static function,
FilterChange.

Just putting that code inline seems just as readable as maintaining
the separate function for it.

SUGGESTION:
static inline bool
FilterChange(LogicalDecodingContext *ctx, XLogRecPtr origptr, TransactionId xid,
RelFileLocator *target_locator, ReorderBufferChangeType
change_type)
{
return
ctx->callbacks.filter_change_cb &&
ctx->reorder->try_to_filter_change &&
ReorderBufferFilterByRelFileLocator(ctx->reorder, xid, origptr,
target_locator,
change_type));
}

======
src/backend/replication/logical/reorderbuffer.c

RelFileLocatorCacheInvalidateCallback:

2.
  if (hash_search(RelFileLocatorFilterCache,
- &entry->key,
- HASH_REMOVE,
- NULL) == NULL)
+ &entry->key,
+ HASH_REMOVE,
+ NULL) == NULL)
  elog(ERROR, "hash table corrupted");

I think this whitespace change belongs back in patch 0001 where this
function was introduced, not here in patch 0002.

~~~

ReorderBufferFilterByRelFileLocator:

3.
+ /*
+ * Quick return if we already know that the relation is not to be
+ * decoded. These are for special relations that are unlogged and for
+ * sequences and catalogs.
+ */
+ if (entry->filterable)
+ return true;

/These are for/This is for/

~~~

4.
  if (RelationIsValid(relation))
  {
- entry->relid = RelationGetRelid(relation);
+ if (IsToastRelation(relation))
+ {
+ char   *toast_name = RelationGetRelationName(relation);
+ int     n PG_USED_FOR_ASSERTS_ONLY;
+
+ n = sscanf(toast_name, "pg_toast_%u", &entry->relid);
+
+ Assert(n == 1);
+ }
+ else
+ entry->relid = RelationGetRelid(relation);
+
  entry->filterable = false;
+ rb->try_to_filter_change = rb->filter_change(rb, entry->relid, change_type,
+   true, &cache_valid);
  RelationClose(relation);
  }
  else
  {
  entry->relid = InvalidOid;
- entry->filterable = true;
+ rb->try_to_filter_change = entry->filterable = true;
  }

rb->try_to_filter_change = entry->filterable;
~

Something seems fishy here. AFAICT, the rb->try_to_filter_change will
already be assigned in both the *if* and the *else* blocks. So, why is
it being overwritten again outside that if/else?

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#47Peter Smith
smithpb2250@gmail.com
In reply to: Peter Smith (#43)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin.

Here are some v17-0003 review comments (aka some v16-0003 comments
that were not yet addressed or rejected)

On Fri, Apr 11, 2025 at 4:05 PM Peter Smith <smithpb2250@gmail.com> wrote:
...

======
Commit message

2. missing commit message

Not yet addressed.

~~~

8.
# Insert, delete, and update tests for restricted publication tables
$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "INSERT INTO insert_only_table
VALUES (1, 'to be inserted')");
$node_publisher->safe_psql('postgres', "UPDATE insert_only_table SET
data = 'updated' WHERE id = 1");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering UPDATE/,
'unpublished UPDATE is filtered');

$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "DELETE FROM insert_only_table
WHERE id = 1");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering DELETE/,
'unpublished DELETE is filtered');

$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "INSERT INTO delete_only_table
VALUES (1, 'to be deleted')");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering INSERT/,
'unpublished INSERT is filtered');

~

...

8b.
Change the comment or rearrange the tests so they are in the same
order as described

The comment was changed, and now says "Insert, update, and delete
tests ..." but still, the "Filtering INSERT" test is last. IMO, that
test should come first to match the comment.

~

8c.
Looking at the expected logs I wondered if it might be nicer for the
pgoutput_filter_change doing this logging to also emit the relation
name.

Not yet addressed

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#48Ajin Cherian
itsajin@gmail.com
In reply to: Peter Smith (#47)
3 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Tue, Apr 22, 2025 at 5:00 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v17-0001

======
Commit message

1.
Additionally, transactions containing WAL records (INTERNAL_SNAPSHOT,
COMMAND_ID, or INVALIDATION) that modify the historical snapshot
constructed during logical decoding are deemed unfilterable. This
is necessary because constructing a correct historical snapshot
for searching publication information requires processing these WAL record.

~

/these WAL record./these WAL records./

Fixed.

======
src/backend/replication/logical/reorderbuffer.c

ReorderBufferFilterByRelFileLocator:

2.
+ if (found)
+ {
+ rb->try_to_filter_change = entry->filterable;
+ return entry->filterable;
+ }
+

Bad indentation.

Fixed.

======
src/include/replication/reorderbuffer.h

3.
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
+

Why is this extern here? This function is not implemented in patch 0001.

Fixed.

On Wed, Apr 23, 2025 at 1:11 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin,

Some review comments for patch v17-0002.

======
src/backend/replication/logical/decode.c

1.
/*
+ * Check if filtering changes before decoding is supported and we're
not suppressing filter
+ * changes currently.
+ */
+static inline bool
+FilterChangeIsEnabled(LogicalDecodingContext *ctx)
+{
+ return (ctx->callbacks.filter_change_cb != NULL &&
+ ctx->reorder->try_to_filter_change);
+}
+

I still have doubts about the need for/benefits of this to be a
separate function.

It is only called from *one* place within the other static function,
FilterChange.

Just putting that code inline seems just as readable as maintaining
the separate function for it.

SUGGESTION:
static inline bool
FilterChange(LogicalDecodingContext *ctx, XLogRecPtr origptr, TransactionId xid,
RelFileLocator *target_locator, ReorderBufferChangeType
change_type)
{
return
ctx->callbacks.filter_change_cb &&
ctx->reorder->try_to_filter_change &&
ReorderBufferFilterByRelFileLocator(ctx->reorder, xid, origptr,
target_locator,
change_type));
}

Fixed.

======
src/backend/replication/logical/reorderbuffer.c

RelFileLocatorCacheInvalidateCallback:

2.
if (hash_search(RelFileLocatorFilterCache,
- &entry->key,
- HASH_REMOVE,
- NULL) == NULL)
+ &entry->key,
+ HASH_REMOVE,
+ NULL) == NULL)
elog(ERROR, "hash table corrupted");

I think this whitespace change belongs back in patch 0001 where this
function was introduced, not here in patch 0002.

Fixed.

~~~

ReorderBufferFilterByRelFileLocator:

3.
+ /*
+ * Quick return if we already know that the relation is not to be
+ * decoded. These are for special relations that are unlogged and for
+ * sequences and catalogs.
+ */
+ if (entry->filterable)
+ return true;

/These are for/This is for/

Fixed.

~~~

4.
if (RelationIsValid(relation))
{
- entry->relid = RelationGetRelid(relation);
+ if (IsToastRelation(relation))
+ {
+ char   *toast_name = RelationGetRelationName(relation);
+ int     n PG_USED_FOR_ASSERTS_ONLY;
+
+ n = sscanf(toast_name, "pg_toast_%u", &entry->relid);
+
+ Assert(n == 1);
+ }
+ else
+ entry->relid = RelationGetRelid(relation);
+
entry->filterable = false;
+ rb->try_to_filter_change = rb->filter_change(rb, entry->relid, change_type,
+   true, &cache_valid);
RelationClose(relation);
}
else
{
entry->relid = InvalidOid;
- entry->filterable = true;
+ rb->try_to_filter_change = entry->filterable = true;
}

rb->try_to_filter_change = entry->filterable;
~

Something seems fishy here. AFAICT, the rb->try_to_filter_change will
already be assigned in both the *if* and the *else* blocks. So, why is
it being overwritten again outside that if/else?

Fixed.

On Wed, Apr 23, 2025 at 1:49 PM Peter Smith <smithpb2250@gmail.com> wrote:

Hi Ajin.

Here are some v17-0003 review comments (aka some v16-0003 comments
that were not yet addressed or rejected)

On Fri, Apr 11, 2025 at 4:05 PM Peter Smith <smithpb2250@gmail.com> wrote:
...

======
Commit message

2. missing commit message

Not yet addressed.

Fixed.

~~~

8.
# Insert, delete, and update tests for restricted publication tables
$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "INSERT INTO insert_only_table
VALUES (1, 'to be inserted')");
$node_publisher->safe_psql('postgres', "UPDATE insert_only_table SET
data = 'updated' WHERE id = 1");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering UPDATE/,
'unpublished UPDATE is filtered');

$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "DELETE FROM insert_only_table
WHERE id = 1");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering DELETE/,
'unpublished DELETE is filtered');

$log_location = -s $node_publisher->logfile;
$node_publisher->safe_psql('postgres', "INSERT INTO delete_only_table
VALUES (1, 'to be deleted')");
$logfile = slurp_file($node_publisher->logfile, $log_location);
ok($logfile =~ qr/Filtering INSERT/,
'unpublished INSERT is filtered');

~

...

8b.
Change the comment or rearrange the tests so they are in the same
order as described

The comment was changed, and now says "Insert, update, and delete
tests ..." but still, the "Filtering INSERT" test is last. IMO, that
test should come first to match the comment.

~

Fixed.

8c.
Looking at the expected logs I wondered if it might be nicer for the
pgoutput_filter_change doing this logging to also emit the relation
name.

pgoutupt_filter does not have relation name, I have added a debug
message in reorderbuffer.c and modified the test accordingly.

Updated patchset v18 addressing review comments.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

v18-0001-Filter-transactions-that-need-not-be-published.patchapplication/octet-stream; name=v18-0001-Filter-transactions-that-need-not-be-published.patchDownload
From 49cc62e6589f8717db622657a5e57231985b1441 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Wed, 30 Apr 2025 05:57:36 -0400
Subject: [PATCH v18 1/3] Filter transactions that need not be published.

This patch set aims to filter transactions at the decode stage rather than
streaming time, which allows the system to skip processing transactions that do
not contain tables included in the logical replication walsender's publication
list. As a result, this can prevent the serialization of unnecessary records to
disk during non-streaming mode, especially in large transactions, and also
reduce memory and CPU usage in streaming mode when many changes can be filtered
out.

A hash table of relation file locators is implemented to cache whether a
relation is filterable or not. This ensures that we only need to retrieve
relation and publication information from the catalog when a cache entry is
invalid, avoiding the overhead of starting a transaction for each record.

Filtering is temporarily suspended for a sequence of changes (set to 100
changes) when an unfilterable change is encountered. This strategy minimizes
the overhead of hash searching for every record, which is beneficial when the
majority of tables in an instance are published and thus unfilterable. The
threshold of 100 was determined to be the optimal balance based on performance
tests.

Additionally, transactions containing WAL records (INTERNAL_SNAPSHOT,
COMMAND_ID, or INVALIDATION) that modify the historical snapshot
constructed during logical decoding are deemed unfilterable. This
is necessary because constructing a correct historical snapshot
for searching publication information requires processing these WAL records.

Note that this patch filters changes only for system catalog relations,
non-logically logged relations, or temporary heaps and sequences. A subsequent
patch will introduce a new output plugin in pgoutput, which will further filter
changes for relations not included in publications.
---
 src/backend/replication/logical/decode.c        |   5 +
 src/backend/replication/logical/reorderbuffer.c | 472 ++++++++++++++++++++----
 src/include/replication/reorderbuffer.h         |  17 +
 src/tools/pgindent/typedefs.list                |   2 +
 4 files changed, 427 insertions(+), 69 deletions(-)

diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index cc03f07..261774b 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -928,6 +928,11 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	if (ctx->reorder->try_to_filter_change &&
+			ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
+												buf->origptr, &target_locator))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	if (!(xlrec->flags & XLH_INSERT_IS_SPECULATIVE))
 		change->action = REORDER_BUFFER_CHANGE_INSERT;
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 6765511..c7e063a 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -82,6 +82,33 @@
  *	  a bit more memory to the oldest subtransactions, because it's likely
  *	  they are the source for the next sequence of changes.
  *
+ *	  We also try to filter changes at the decode stage rather than at
+ *	  streaming time, which allows the system to skip processing txns that
+ *	  do not contain tables that need to be decoded. As a result,
+ *	  this can prevent the serialization of unnecessary records to disk
+ *	  during non-streaming mode, especially in large transactions, and also
+ *	  reduce memory and CPU usage in streaming mode when many changes can be
+ *	  filtered out.
+ *
+ *	  A hash table of relation file locators is implemented to cache whether a
+ *	  relation is filterable or not. This ensures that we only need to retrieve
+ *	  relation and publication information from the catalog when a cache entry
+ *	  is invalid, avoiding the overhead of starting a transaction for each
+ *	  record.
+ *
+ *	  Filtering is temporarily suspended for a sequence of changes (set to 100
+ *	  changes) when an unfilterable change is encountered. This strategy
+ *	  minimizes the overhead of hash searching for every record, which is
+ *    beneficial when the majority of tables in an instance are published and
+ *	  thus unfilterable. The threshold of 100 was determined to be the optimal
+ *	  balance based on performance tests.
+ *
+ *	  Additionally, transactions containing WAL records (INTERNAL_SNAPSHOT,
+ *	  COMMAND_ID, or INVALIDATION) that modify the historical snapshot
+ *	  constructed during logical decoding are deemed unfilterable. This is
+ *	  necessary because constructing a correct historical snapshot for
+ *	  searching publication information requires processing these WAL records.
+ *
  * -------------------------------------------------------------------------
  */
 #include "postgres.h"
@@ -109,6 +136,7 @@
 #include "storage/procarray.h"
 #include "storage/sinval.h"
 #include "utils/builtins.h"
+#include "utils/inval.h"
 #include "utils/memutils.h"
 #include "utils/rel.h"
 #include "utils/relfilenumbermap.h"
@@ -227,8 +255,10 @@ static ReorderBufferTXN *ReorderBufferTXNByXid(ReorderBuffer *rb,
 											   XLogRecPtr lsn, bool create_as_top);
 static void ReorderBufferTransferSnapToParent(ReorderBufferTXN *txn,
 											  ReorderBufferTXN *subtxn);
-
 static void AssertTXNLsnOrder(ReorderBuffer *rb);
+static Relation GetDecodableRelation(ReorderBuffer *rb,
+										 RelFileLocator *rlocator,
+										 bool has_tuple);
 
 /* ---------------------------------------
  * support functions for lsn-order iterating over the ->changes of a
@@ -279,6 +309,8 @@ static Snapshot ReorderBufferCopySnap(ReorderBuffer *rb, Snapshot orig_snap,
  */
 static inline bool ReorderBufferCanStream(ReorderBuffer *rb);
 static inline bool ReorderBufferCanStartStreaming(ReorderBuffer *rb);
+static Snapshot ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb,
+											   ReorderBufferTXN *txn);
 static void ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn);
 static void ReorderBufferStreamCommit(ReorderBuffer *rb, ReorderBufferTXN *txn);
 
@@ -305,6 +337,50 @@ static void ReorderBufferChangeMemoryUpdate(ReorderBuffer *rb,
 											bool addition, Size sz);
 
 /*
+ * ---------------------------------------
+ * RelFileLocator filtering
+ * ---------------------------------------
+ * This hash table serves as a lookup table for determining if a relation can
+ * be filtered before being decoded and queued into the buffer.
+ *
+ * The hash table shares the same lifespan as the reorder buffer. This is
+ * crucial because each reorderbuffer may have different configurations or be
+ * associated with different output plugins, affecting the types of tables to
+ * be processed.
+ */
+
+static HTAB *RelFileLocatorFilterCache = NULL;
+
+static bool relation_callbacks_registered = false;
+
+typedef struct ReorderBufferRelFileLocatorKey
+{
+	Oid			reltablespace;
+	RelFileNumber relfilenumber;
+} ReorderBufferRelFileLocatorKey;
+
+/* Hash table entry used to determine if the relation can be filtered. */
+typedef struct ReorderBufferRelFileLocatorEnt
+{
+	ReorderBufferRelFileLocatorKey key;
+	Oid			relid;
+	bool		filterable; /* if true, belongs to a relation that can be filtered */
+} ReorderBufferRelFileLocatorEnt;
+
+static void RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid);
+static void ReorderBufferMemoryResetcallback(void *arg);
+
+/*
+ * After encountering a change that cannot be filtered out, filtering is
+ * temporarily suspended. Filtering resumes after processing the next
+ * CHANGES_THRESHOLD_FOR_FILTER changes.
+ * This strategy helps to minimize the overhead of performing a
+ * hash table search for each record, especially when most changes are not
+ * filterable.
+ */
+#define CHANGES_THRESHOLD_FOR_FILTER 100
+
+/*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
  * prior ReorderBuffer instances for the same slot.
  */
@@ -362,6 +438,34 @@ ReorderBufferAllocate(void)
 	buffer->by_txn = hash_create("ReorderBufferByXid", 1000, &hash_ctl,
 								 HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
 
+	/*
+	 * To support early filtering of changes, this hash table acts
+	 * as a lookup table to determine if the corresponding relation is
+	 * required to be decoded and queued into the buffer. The hash table
+	 * shares the same lifespan as the reorder buffer.
+	 *
+	 * Also setup the callback to invalidate cache when relations are updated.
+	 */
+	hash_ctl.keysize = sizeof(ReorderBufferRelFileLocatorKey);
+	hash_ctl.entrysize = sizeof(ReorderBufferRelFileLocatorEnt);
+	hash_ctl.hcxt = buffer->context;
+
+	RelFileLocatorFilterCache =
+		hash_create("RelFileLocatorFilterCache", 1000, &hash_ctl,
+					HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
+
+	buffer->relfile_callback.arg = NULL;
+	buffer->relfile_callback.func = ReorderBufferMemoryResetcallback;
+	MemoryContextRegisterResetCallback(buffer->context, &buffer->relfile_callback);
+
+	if (!relation_callbacks_registered)
+	{
+		/* Watch for invalidation events. */
+		CacheRegisterRelcacheCallback(RelFileLocatorCacheInvalidateCallback,
+									  (Datum) 0);
+		relation_callbacks_registered = true;
+	}
+
 	buffer->by_txn_last_xid = InvalidTransactionId;
 	buffer->by_txn_last_txn = NULL;
 
@@ -372,6 +476,9 @@ ReorderBufferAllocate(void)
 	/* txn_heap is ordered by transaction size */
 	buffer->txn_heap = pairingheap_allocate(ReorderBufferTXNSizeCompare, NULL);
 
+	buffer->try_to_filter_change = true;
+	buffer->unfiltered_changes_count = 0;
+
 	buffer->spillTxns = 0;
 	buffer->spillCount = 0;
 	buffer->spillBytes = 0;
@@ -826,6 +933,14 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 
 		toptxn->txn_flags |= RBTXN_HAS_STREAMABLE_CHANGE;
 	}
+	else if (change->action == REORDER_BUFFER_CHANGE_INVALIDATION ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_SNAPSHOT ||
+			 change->action == REORDER_BUFFER_CHANGE_INTERNAL_COMMAND_ID)
+	{
+		ReorderBufferTXN *toptxn = rbtxn_get_toptxn(txn);
+
+		toptxn->txn_flags |= RBTXN_HAS_SNAPSHOT_CHANGES;
+	}
 
 	change->lsn = lsn;
 	change->txn = txn;
@@ -835,6 +950,17 @@ ReorderBufferQueueChange(ReorderBuffer *rb, TransactionId xid, XLogRecPtr lsn,
 	txn->nentries++;
 	txn->nentries_mem++;
 
+	/*
+	 * If filtering is currently suspended and we've crossed the change threshold,
+	 * attempt to filter again
+	 */
+	if (!rb->try_to_filter_change && (++rb->unfiltered_changes_count
+									>= CHANGES_THRESHOLD_FOR_FILTER))
+	{
+		rb->try_to_filter_change = true;
+		rb->unfiltered_changes_count = 0;
+	}
+
 	/* update memory accounting information */
 	ReorderBufferChangeMemoryUpdate(rb, change, NULL, true,
 									ReorderBufferChangeSize(change));
@@ -1734,6 +1860,9 @@ ReorderBufferTruncateTXN(ReorderBuffer *rb, ReorderBufferTXN *txn, bool txn_prep
 		txn->txn_flags |= RBTXN_IS_SERIALIZED_CLEAR;
 	}
 
+	/* All snapshot changes up to this point have been processed. */
+	txn->txn_flags &= ~RBTXN_HAS_SNAPSHOT_CHANGES;
+
 	/* also reset the number of entries in the transaction */
 	txn->nentries_mem = 0;
 	txn->nentries = 0;
@@ -2177,6 +2306,85 @@ ReorderBufferResetTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 }
 
 /*
+ * Return the relation corresponding to the given RelFileLocator 'rlocator' if
+ * relevant to decoding. Otherwise, return NULL.
+ *
+ * We don't decode catalog files, relations that are not logically logged,
+ * temporary heaps and sequences.
+ */
+static Relation
+GetDecodableRelation(ReorderBuffer *rb, RelFileLocator *rlocator,
+						 bool has_tuple)
+{
+	bool		filterable = false;
+	Relation	relation;
+	Oid			reloid;
+	RelPathStr  path;
+
+	reloid = RelidByRelfilenumber(rlocator->spcOid, rlocator->relNumber);
+	path = relpathperm(*rlocator, MAIN_FORKNUM);
+
+	if (!OidIsValid(reloid))
+	{
+		if (!has_tuple)
+		{
+			/*
+			 * Mapped catalog tuple without data, emitted while catalog table
+			 * was in the process of being rewritten. We can fail to look up
+			 * the relfilenumber, because the relmapper has no "historic" view,
+			 * in contrast to the normal catalog during decoding. Thus repeated
+			 * rewrites can cause a lookup failure. That's OK because we do not
+			 * decode catalog changes anyway. Normally such tuples would be
+			 * skipped over below, but we can't identify whether the table
+			 * should be logically logged without mapping the relfilenumber
+			 * to the oid.
+			 */
+			return NULL;
+		}
+
+		elog(ERROR, "could not map filenumber \"%s\" to relation OID",
+				path.str);
+	}
+
+	relation = RelationIdGetRelation(reloid);
+
+	if (!RelationIsValid(relation))
+		elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
+			 reloid, path.str);
+
+	if (!RelationIsLogicallyLogged(relation))
+		filterable = true;
+
+	else if (relation->rd_rel->relrewrite && !rb->output_rewrites)
+	{
+		/*
+		 * Ignore temporary heaps created during DDL unless the plugin has asked
+		 * for them.
+		 */
+		filterable = true;
+	}
+
+	else if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
+	{
+		/*
+		 * For now ignore sequence changes entirely. Most of the time they don't
+		 * log changes using records we understand, so it doesn't make sense to
+		 * handle the few cases we do.
+		 */
+		filterable = true;
+	}
+
+	/* Return NULL to indicate that this relation need not be decoded. */
+	if (filterable)
+	{
+		RelationClose(relation);
+		return NULL;
+	}
+
+	return relation;
+}
+
+/*
  * Helper function for ReorderBufferReplay and ReorderBufferStreamTXN.
  *
  * Send data of a transaction (and its subtransactions) to the
@@ -2248,7 +2456,6 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 		while ((change = ReorderBufferIterTXNNext(rb, iterstate)) != NULL)
 		{
 			Relation	relation = NULL;
-			Oid			reloid;
 
 			CHECK_FOR_INTERRUPTS();
 
@@ -2307,55 +2514,10 @@ ReorderBufferProcessTXN(ReorderBuffer *rb, ReorderBufferTXN *txn,
 				case REORDER_BUFFER_CHANGE_DELETE:
 					Assert(snapshot_now);
 
-					reloid = RelidByRelfilenumber(change->data.tp.rlocator.spcOid,
-												  change->data.tp.rlocator.relNumber);
-
-					/*
-					 * Mapped catalog tuple without data, emitted while
-					 * catalog table was in the process of being rewritten. We
-					 * can fail to look up the relfilenumber, because the
-					 * relmapper has no "historic" view, in contrast to the
-					 * normal catalog during decoding. Thus repeated rewrites
-					 * can cause a lookup failure. That's OK because we do not
-					 * decode catalog changes anyway. Normally such tuples
-					 * would be skipped over below, but we can't identify
-					 * whether the table should be logically logged without
-					 * mapping the relfilenumber to the oid.
-					 */
-					if (reloid == InvalidOid &&
-						change->data.tp.newtuple == NULL &&
-						change->data.tp.oldtuple == NULL)
-						goto change_done;
-					else if (reloid == InvalidOid)
-						elog(ERROR, "could not map filenumber \"%s\" to relation OID",
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM).str);
-
-					relation = RelationIdGetRelation(reloid);
-
+					relation = GetDecodableRelation(rb, &change->data.tp.rlocator,
+														(change->data.tp.newtuple != NULL ||
+														 change->data.tp.oldtuple != NULL));
 					if (!RelationIsValid(relation))
-						elog(ERROR, "could not open relation with OID %u (for filenumber \"%s\")",
-							 reloid,
-							 relpathperm(change->data.tp.rlocator,
-										 MAIN_FORKNUM).str);
-
-					if (!RelationIsLogicallyLogged(relation))
-						goto change_done;
-
-					/*
-					 * Ignore temporary heaps created during DDL unless the
-					 * plugin has asked for them.
-					 */
-					if (relation->rd_rel->relrewrite && !rb->output_rewrites)
-						goto change_done;
-
-					/*
-					 * For now ignore sequence changes entirely. Most of the
-					 * time they don't log changes using records we
-					 * understand, so it doesn't make sense to handle the few
-					 * cases we do.
-					 */
-					if (relation->rd_rel->relkind == RELKIND_SEQUENCE)
 						goto change_done;
 
 					/* user-triggered change */
@@ -4171,19 +4333,13 @@ ReorderBufferCanStartStreaming(ReorderBuffer *rb)
 }
 
 /*
- * Send data of a large transaction (and its subtransactions) to the
- * output plugin, but using the stream API.
+ * This function generates or retrieves a consistent snapshot of a transaction
+ * that is currently in progress for logical replication.
  */
-static void
-ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+static Snapshot
+ReorderBufferStreamTXNSnapshot(ReorderBuffer *rb, ReorderBufferTXN *txn)
 {
 	Snapshot	snapshot_now;
-	CommandId	command_id;
-	Size		stream_bytes;
-	bool		txn_is_streamed;
-
-	/* We can never reach here for a subtransaction. */
-	Assert(rbtxn_is_toptxn(txn));
 
 	/*
 	 * We can't make any assumptions about base snapshot here, similar to what
@@ -4226,12 +4382,11 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		if (txn->base_snapshot == NULL)
 		{
 			Assert(txn->ninvalidations == 0);
-			return;
+			return NULL;
 		}
 
-		command_id = FirstCommandId;
 		snapshot_now = ReorderBufferCopySnap(rb, txn->base_snapshot,
-											 txn, command_id);
+											 txn, FirstCommandId);
 	}
 	else
 	{
@@ -4244,17 +4399,40 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 		 * the LSN condition in the previous branch (so no need to walk
 		 * through subxacts again). In fact, we must not do that as we may be
 		 * using snapshot half-way through the subxact.
-		 */
-		command_id = txn->command_id;
-
-		/*
+		 *
 		 * We can't use txn->snapshot_now directly because after the last
 		 * streaming run, we might have got some new sub-transactions. So we
 		 * need to add them to the snapshot.
 		 */
 		snapshot_now = ReorderBufferCopySnap(rb, txn->snapshot_now,
-											 txn, command_id);
+											 txn, txn->command_id);
+	}
+
+	return snapshot_now;
+}
+
+/*
+ * Send data of a large transaction (and its subtransactions) to the
+ * output plugin, but using the stream API.
+ */
+static void
+ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
+{
+	Snapshot	snapshot_now;
+	Size		stream_bytes;
+	bool		txn_is_streamed;
+
+	/* We can never reach here for a subtransaction. */
+	Assert(rbtxn_is_toptxn(txn));
+
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, txn);
+
+	/* This transaction didn't make any changes to the database till now. */
+	if (snapshot_now == NULL)
+		return;
 
+	if (txn->snapshot_now != NULL)
+	{
 		/* Free the previously copied snapshot. */
 		Assert(txn->snapshot_now->copied);
 		ReorderBufferFreeSnap(rb, txn->snapshot_now);
@@ -4272,7 +4450,7 @@ ReorderBufferStreamTXN(ReorderBuffer *rb, ReorderBufferTXN *txn)
 
 	/* Process and send the changes to output plugin. */
 	ReorderBufferProcessTXN(rb, txn, InvalidXLogRecPtr, snapshot_now,
-							command_id, true);
+							snapshot_now->curcid, true);
 
 	rb->streamCount += 1;
 	rb->streamBytes += stream_bytes;
@@ -5462,6 +5640,162 @@ restart:
 }
 
 /*
+ * Flush mapping entries when the corresponding relations are updated in pg_class.
+ * If relid is InvalidOid then do a complete reset of the RelFileLocatorFilterCache.
+ */
+static void
+RelFileLocatorCacheInvalidateCallback(Datum arg, Oid relid)
+{
+	HASH_SEQ_STATUS status;
+	ReorderBufferRelFileLocatorEnt *entry;
+
+	if (!RelFileLocatorFilterCache)
+		return;
+
+	hash_seq_init(&status, RelFileLocatorFilterCache);
+
+	/* Slightly inefficient algorithm but not performance critical path */
+	while ((entry = (ReorderBufferRelFileLocatorEnt *) hash_seq_search(&status)) != NULL)
+	{
+		/*
+		 * If a complete reset is requested (when relid parameter is InvalidOid),
+		 * we must remove all entries, otherwise just remove the specific relation's
+		 * entry. Remove any invalid cache entries (these are indicated by invalid
+		 * entry->relid)
+		 */
+		if (relid == InvalidOid ||  /* complete reset */
+			entry->relid == InvalidOid ||   /* invalid cache entry */
+			entry->relid == relid)  /* individual flushed relation */
+		{
+			if (hash_search(RelFileLocatorFilterCache,
+							&entry->key,
+							HASH_REMOVE,
+							NULL) == NULL)
+				elog(ERROR, "hash table corrupted");
+		}
+	}
+}
+
+/*
+ * Context reset/delete callback for RelFileLocatorFilterCache.
+ */
+static void
+ReorderBufferMemoryResetcallback(void *arg)
+{
+	RelFileLocatorFilterCache = NULL;
+}
+
+/*
+ * Determine whether the record corresponding to the relation identified by
+ * 'rlocator' needs to be filtered and not decoded and queued in the buffer.
+ *
+ * Determining the necessity of decoding requires accessing relation
+ * information from the system catalog, which requires a historical snapshot.
+ * We cannot directly use the current snapshot of the transaction due to the
+ * presence of INTERNAL_SNAPSHOT, COMMAND_ID, or INVALIDATION records in the
+ * buffer that could modify the snapshot. A proper snapshot can only be
+ * constructed after these records are processed in ReorderBufferProcessTXN, or
+ * if we are decoding a transaction without these records. See comment on top
+ * of GetDecodableRelation() to see the kinds of relations that are not
+ * decoded by the reorderbuffer.
+ *
+ * To reduce the overhead from the system catalog access and transaction
+ * management, the results are cached in the 'RelFileLocatorFilterCache' hash
+ * table.
+ *
+ * Returns true if the relation can be filtered; otherwise, false.
+ */
+bool
+ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+									 XLogRecPtr lsn, RelFileLocator *rlocator)
+
+{
+	bool		found;
+	Relation	relation;
+	bool		using_subtxn;
+	Snapshot	snapshot_now;
+	ReorderBufferTXN *txn,
+					 *toptxn;
+	ReorderBufferRelFileLocatorEnt *entry;
+	ReorderBufferRelFileLocatorKey key;
+
+	Assert(RelFileLocatorFilterCache);
+
+	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
+	Assert(txn);
+	toptxn = rbtxn_get_toptxn(txn);
+	rb->try_to_filter_change = false;
+
+	/*
+	 * We cannot construct an accurate historical snapshot until all pending
+	 * records in this transaction that might update the snapshot are
+	 * processed.
+	 */
+	if (rbtxn_has_snapshot_changes(toptxn))
+		return false;
+
+	key.reltablespace = rlocator->spcOid;
+	key.relfilenumber = rlocator->relNumber;
+	entry = hash_search(RelFileLocatorFilterCache, &key, HASH_ENTER, &found);
+
+	if (found)
+	{
+		rb->try_to_filter_change = entry->filterable;
+		return entry->filterable;
+	}
+
+	/* constructs a temporary historical snapshot */
+	snapshot_now = ReorderBufferStreamTXNSnapshot(rb, toptxn);
+	Assert(snapshot_now);
+
+	/* build data to be able to lookup the CommandIds of catalog tuples */
+	ReorderBufferBuildTupleCidHash(rb, toptxn);
+
+	/* setup the initial snapshot */
+	SetupHistoricSnapshot(snapshot_now, toptxn->tuplecid_hash);
+
+	using_subtxn = IsTransactionOrTransactionBlock();
+
+	if (using_subtxn)
+		BeginInternalSubTransaction("filter change by RelFileLocator");
+	else
+		StartTransactionCommand();
+
+	relation = GetDecodableRelation(rb, rlocator, true);
+
+	if (RelationIsValid(relation))
+	{
+		entry->relid = RelationGetRelid(relation);
+		entry->filterable = false;
+		RelationClose(relation);
+	}
+	else
+	{
+		entry->relid = InvalidOid;
+		entry->filterable = true;
+	}
+
+	rb->try_to_filter_change = entry->filterable;
+
+	ReorderBufferFreeSnap(rb, snapshot_now);
+
+	TeardownHistoricSnapshot(false);
+
+	/*
+	 * Aborting the current (sub-)transaction as a whole has the right
+	 * semantics. We want all locks acquired in here to be released, not
+	 * reassigned to the parent and we do not want any database access have
+	 * persistent effects.
+	 */
+	AbortCurrentTransaction();
+
+	if (using_subtxn)
+		RollbackAndReleaseCurrentSubTransaction();
+
+	return entry->filterable;
+}
+
+/*
  * Count invalidation messages of specified transaction.
  *
  * Returns number of messages, and msgs is set to the pointer of the linked
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 24e88c4..363d9cd 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -176,6 +176,7 @@ typedef struct ReorderBufferChange
 #define RBTXN_SENT_PREPARE			0x0200
 #define RBTXN_IS_COMMITTED			0x0400
 #define RBTXN_IS_ABORTED			0x0800
+#define RBTXN_HAS_SNAPSHOT_CHANGES  0x1000
 
 #define RBTXN_PREPARE_STATUS_MASK	(RBTXN_IS_PREPARED | RBTXN_SKIPPED_PREPARE | RBTXN_SENT_PREPARE)
 
@@ -215,6 +216,12 @@ typedef struct ReorderBufferChange
 	((txn)->txn_flags & RBTXN_HAS_STREAMABLE_CHANGE) != 0 \
 )
 
+/* Does this transaction make changes to the current snapshot? */
+#define rbtxn_has_snapshot_changes(txn) \
+( \
+	((txn)->txn_flags & RBTXN_HAS_SNAPSHOT_CHANGES) != 0 \
+)
+
 /*
  * Has this transaction been streamed to downstream?
  *
@@ -641,6 +648,7 @@ struct ReorderBuffer
 	 * Private memory context.
 	 */
 	MemoryContext context;
+	MemoryContextCallback relfile_callback;
 
 	/*
 	 * Memory contexts for specific types objects
@@ -661,6 +669,12 @@ struct ReorderBuffer
 	/* Max-heap for sizes of all top-level and sub transactions */
 	pairingheap *txn_heap;
 
+	/* should we try to filter the change? */
+	bool		try_to_filter_change;
+
+	/* number of changes after a failed attempt at filtering */
+	int8		unfiltered_changes_count;
+
 	/*
 	 * Statistics about transactions spilled to disk.
 	 *
@@ -764,4 +778,7 @@ extern uint32 ReorderBufferGetInvalidations(ReorderBuffer *rb,
 
 extern void StartupReorderBuffer(void);
 
+extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
+												XLogRecPtr lsn, RelFileLocator *rlocator);
+
 #endif
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index e5879e0..35d2628 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2509,6 +2509,8 @@ ReorderBufferIterTXNEntry
 ReorderBufferIterTXNState
 ReorderBufferMessageCB
 ReorderBufferPrepareCB
+ReorderBufferRelFileLocatorEnt
+ReorderBufferRelFileLocatorKey
 ReorderBufferRollbackPreparedCB
 ReorderBufferStreamAbortCB
 ReorderBufferStreamChangeCB
-- 
1.8.3.1

v18-0002-Introduce-an-output-plugin-callback-to-filter-ch.patchapplication/octet-stream; name=v18-0002-Introduce-an-output-plugin-callback-to-filter-ch.patchDownload
From 5f19b17c12ddcc6615afd8a9fdf435826b80d9e3 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Wed, 30 Apr 2025 06:23:03 -0400
Subject: [PATCH v18 2/3] Introduce an output plugin callback to filter changes

This new output plugin callback provides an option to logical decoding plugins to filter out
changes early. The primary purpose of the callback is to conserve memory and processing cycles
by excluding changes that are not required by output plugins.
---
 doc/src/sgml/logicaldecoding.sgml               | 41 ++++++++++++++-
 src/backend/replication/logical/decode.c        | 38 ++++++++++++--
 src/backend/replication/logical/logical.c       | 41 +++++++++++++++
 src/backend/replication/logical/reorderbuffer.c | 57 +++++++++++++++-----
 src/backend/replication/pgoutput/pgoutput.c     | 70 +++++++++++++++++++++++++
 src/include/replication/output_plugin.h         | 20 +++++++
 src/include/replication/reorderbuffer.h         | 12 ++++-
 src/test/subscription/t/013_partition.pl        |  7 +++
 8 files changed, 267 insertions(+), 19 deletions(-)

diff --git a/doc/src/sgml/logicaldecoding.sgml b/doc/src/sgml/logicaldecoding.sgml
index 3f2bcd4..788fe05 100644
--- a/doc/src/sgml/logicaldecoding.sgml
+++ b/doc/src/sgml/logicaldecoding.sgml
@@ -560,6 +560,7 @@ typedef struct OutputPluginCallbacks
     LogicalDecodeCommitCB commit_cb;
     LogicalDecodeMessageCB message_cb;
     LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+    LogicalDecodeFilterChangeCB filter_change_cb;
     LogicalDecodeShutdownCB shutdown_cb;
     LogicalDecodeFilterPrepareCB filter_prepare_cb;
     LogicalDecodeBeginPrepareCB begin_prepare_cb;
@@ -582,8 +583,8 @@ typedef void (*LogicalOutputPluginInit) (struct OutputPluginCallbacks *cb);
      and <function>commit_cb</function> callbacks are required,
      while <function>startup_cb</function>, <function>truncate_cb</function>,
      <function>message_cb</function>, <function>filter_by_origin_cb</function>,
-     and <function>shutdown_cb</function> are optional.
-     If <function>truncate_cb</function> is not set but a
+     <function>shutdown_cb</function>, and <function>filter_change_cb</function>
+     are optional. If <function>truncate_cb</function> is not set but a
      <command>TRUNCATE</command> is to be decoded, the action will be ignored.
     </para>
 
@@ -871,6 +872,42 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
      </para>
      </sect3>
 
+     <sect3 id="logicaldecoding-output-plugin-filter-change">
+     <title>Change Filter Callback</title>
+
+     <para>
+      The optional <function>filter_change_cb</function> is called before a
+      change record is decoded to determine whether the change can be filtered
+      out.
+<programlisting>
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+                                             Oid relid,
+                                             ReorderBufferChangeType change_type,
+                                             bool in_txn, bool *cache_valid);
+</programlisting>
+      To indicate that decoding can be skipped for the given
+      <parameter>change_type</parameter>, return <literal>true</literal>;
+      <literal>false</literal> otherwise.
+      The <parameter>in_txn</parameter> parameter indicates whether the
+      callback is invoked within a transaction block.
+      When <parameter>in_txn</parameter> is false, and if making a decision to filter a change requires being inside a
+      transaction block, such as needing access to the catalog, set
+      <parameter>*cache_valid</parameter> to <literal>false</literal>.
+      This ensures that the callback will be reinvoked once a transaction block
+      starts. If a decision can be made immediately, set
+      <parameter>*cache_valid</parameter> to <literal>true</literal>.
+     </para>
+     <para>
+      The primary purpose of this callback function is to optimize memory usage
+      and processing efficiency by filtering out changes that are unnecessary for
+      output plugins. It enables output plugins to selectively process relevant
+      changes. Caching filtering decisions locally is recommended, as it enables
+      the callback to provide cached results without repeatedly initiating
+      transactions or querying the catalog. This approach minimizes overhead
+      and improves efficiency during the decoding process.
+     </para>
+     </sect3>
+
     <sect3 id="logicaldecoding-output-plugin-message">
      <title>Generic Message Callback</title>
 
diff --git a/src/backend/replication/logical/decode.c b/src/backend/replication/logical/decode.c
index 261774b..77c8859 100644
--- a/src/backend/replication/logical/decode.c
+++ b/src/backend/replication/logical/decode.c
@@ -894,6 +894,18 @@ DecodeAbort(LogicalDecodingContext *ctx, XLogRecordBuffer *buf,
 	UpdateDecodingStats(ctx);
 }
 
+/* Function to determine whether to filter the change */
+static inline bool
+FilterChange(LogicalDecodingContext *ctx, XLogRecPtr origptr, TransactionId xid,
+							RelFileLocator *target_locator, ReorderBufferChangeType change_type)
+{
+	return
+		(ctx->callbacks.filter_change_cb &&
+		 ctx->reorder->try_to_filter_change &&
+		 ReorderBufferFilterByRelFileLocator(ctx->reorder, xid, origptr, target_locator,
+											 change_type));
+}
+
 /*
  * Parse XLOG_HEAP_INSERT (not MULTI_INSERT!) records into tuplebufs.
  *
@@ -928,9 +940,9 @@ DecodeInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
-	if (ctx->reorder->try_to_filter_change &&
-			ReorderBufferFilterByRelFileLocator(ctx->reorder, XLogRecGetXid(r),
-												buf->origptr, &target_locator))
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+						REORDER_BUFFER_CHANGE_INSERT))
 		return;
 
 	change = ReorderBufferAllocChange(ctx->reorder);
@@ -983,6 +995,11 @@ DecodeUpdate(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+					   REORDER_BUFFER_CHANGE_UPDATE))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_UPDATE;
 	change->origin_id = XLogRecGetOrigin(r);
@@ -1049,6 +1066,11 @@ DecodeDelete(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+					   REORDER_BUFFER_CHANGE_DELETE))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 
 	if (xlrec->flags & XLH_DELETE_IS_SUPER)
@@ -1151,6 +1173,11 @@ DecodeMultiInsert(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &rlocator,
+					   REORDER_BUFFER_CHANGE_INSERT))
+		return;
+
 	/*
 	 * We know that this multi_insert isn't for a catalog, so the block should
 	 * always have data even if a full-page write of it is taken.
@@ -1242,6 +1269,11 @@ DecodeSpecConfirm(LogicalDecodingContext *ctx, XLogRecordBuffer *buf)
 	if (FilterByOrigin(ctx, XLogRecGetOrigin(r)))
 		return;
 
+	/* Can the relation associated with this change be skipped? */
+	if (FilterChange(ctx, buf->origptr, XLogRecGetXid(r), &target_locator,
+						REORDER_BUFFER_CHANGE_INSERT))
+		return;
+
 	change = ReorderBufferAllocChange(ctx->reorder);
 	change->action = REORDER_BUFFER_CHANGE_INTERNAL_SPEC_CONFIRM;
 	change->origin_id = XLogRecGetOrigin(r);
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index a8d2e02..09ee1df 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -74,6 +74,9 @@ static void truncate_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 static void message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 							   XLogRecPtr message_lsn, bool transactional,
 							   const char *prefix, Size message_size, const char *message);
+static bool filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+								  ReorderBufferChangeType change_type, bool in_txn,
+								  bool *cache_valid);
 
 /* streaming callbacks */
 static void stream_start_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
@@ -220,6 +223,7 @@ StartupDecodingContext(List *output_plugin_options,
 	/* wrap output plugin callbacks, so we can add error context information */
 	ctx->reorder->begin = begin_cb_wrapper;
 	ctx->reorder->apply_change = change_cb_wrapper;
+	ctx->reorder->filter_change = filter_change_cb_wrapper;
 	ctx->reorder->apply_truncate = truncate_cb_wrapper;
 	ctx->reorder->commit = commit_cb_wrapper;
 	ctx->reorder->message = message_cb_wrapper;
@@ -1224,6 +1228,43 @@ filter_by_origin_cb_wrapper(LogicalDecodingContext *ctx, RepOriginId origin_id)
 	return ret;
 }
 
+static bool
+filter_change_cb_wrapper(ReorderBuffer *cache, Oid relid,
+					  ReorderBufferChangeType change_type, bool in_txn,
+					  bool *cache_valid)
+{
+	LogicalDecodingContext *ctx = cache->private_data;
+	LogicalErrorCallbackState state;
+	ErrorContextCallback errcallback;
+	bool		ret;
+
+	Assert(!ctx->fast_forward);
+
+	if (ctx->callbacks.filter_change_cb == NULL)
+		return false;
+
+	/* Push callback + info on the error context stack */
+	state.ctx = ctx;
+	state.callback_name = "filter_change";
+	state.report_location = InvalidXLogRecPtr;
+	errcallback.callback = output_plugin_error_callback;
+	errcallback.arg = (void *) &state;
+	errcallback.previous = error_context_stack;
+	error_context_stack = &errcallback;
+
+	/* set output state */
+	ctx->accept_writes = false;
+	ctx->end_xact = false;
+
+	/* do the actual work: call callback */
+	ret = ctx->callbacks.filter_change_cb(ctx, relid, change_type, in_txn, cache_valid);
+
+	/* Pop the error context stack */
+	error_context_stack = errcallback.previous;
+
+	return ret;
+}
+
 static void
 message_cb_wrapper(ReorderBuffer *cache, ReorderBufferTXN *txn,
 				   XLogRecPtr message_lsn, bool transactional,
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c7e063a..cd2b052 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -5703,17 +5703,19 @@ ReorderBufferMemoryResetcallback(void *arg)
  * management, the results are cached in the 'RelFileLocatorFilterCache' hash
  * table.
  *
- * Returns true if the relation can be filtered; otherwise, false.
+ * Returns true if this change_type can be filtered; otherwise, false.
  */
 bool
 ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
-									 XLogRecPtr lsn, RelFileLocator *rlocator)
+									XLogRecPtr lsn, RelFileLocator *rlocator,
+									ReorderBufferChangeType change_type)
 
 {
-	bool		found;
-	Relation	relation;
-	bool		using_subtxn;
-	Snapshot	snapshot_now;
+	bool        found;
+	bool		cache_valid;
+	Relation    relation;
+	bool        using_subtxn;
+	Snapshot    snapshot_now;
 	ReorderBufferTXN *txn,
 					 *toptxn;
 	ReorderBufferRelFileLocatorEnt *entry;
@@ -5724,7 +5726,6 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	txn = ReorderBufferTXNByXid(rb, xid, true, NULL, lsn, true);
 	Assert(txn);
 	toptxn = rbtxn_get_toptxn(txn);
-	rb->try_to_filter_change = false;
 
 	/*
 	 * We cannot construct an accurate historical snapshot until all pending
@@ -5741,7 +5742,26 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (found)
 	{
 		rb->try_to_filter_change = entry->filterable;
-		return entry->filterable;
+
+		/*
+		 * Quick return if we already know that the relation is not to be
+		 * decoded. This is for special relations that are unlogged and for
+		 * sequences and catalogs.
+		 */
+		if (entry->filterable)
+			return true;
+
+		/* Allow the output plugin to filter relations */
+		rb->try_to_filter_change = rb->filter_change(rb, entry->relid, change_type,
+													 false, &cache_valid);
+
+		/*
+		 * If plugin had the relation ready in cache, the response is valid,
+		 * else we'll need to call the plugin a second time within a
+		 * transaction.
+		 */
+		if (cache_valid)
+			return rb->try_to_filter_change;
 	}
 
 	/* constructs a temporary historical snapshot */
@@ -5765,18 +5785,29 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 
 	if (RelationIsValid(relation))
 	{
-		entry->relid = RelationGetRelid(relation);
+		if (IsToastRelation(relation))
+		{
+			char   *toast_name = RelationGetRelationName(relation);
+			int     n PG_USED_FOR_ASSERTS_ONLY;
+
+			n = sscanf(toast_name, "pg_toast_%u", &entry->relid);
+
+			Assert(n == 1);
+		}
+		else
+			entry->relid = RelationGetRelid(relation);
+
 		entry->filterable = false;
+		rb->try_to_filter_change = rb->filter_change(rb, entry->relid, change_type,
+												  true, &cache_valid);
 		RelationClose(relation);
 	}
 	else
 	{
 		entry->relid = InvalidOid;
-		entry->filterable = true;
+		rb->try_to_filter_change = entry->filterable = true;
 	}
 
-	rb->try_to_filter_change = entry->filterable;
-
 	ReorderBufferFreeSnap(rb, snapshot_now);
 
 	TeardownHistoricSnapshot(false);
@@ -5792,7 +5823,7 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 	if (using_subtxn)
 		RollbackAndReleaseCurrentSubTransaction();
 
-	return entry->filterable;
+	return rb->try_to_filter_change;
 }
 
 /*
diff --git a/src/backend/replication/pgoutput/pgoutput.c b/src/backend/replication/pgoutput/pgoutput.c
index 693a766..c356423 100644
--- a/src/backend/replication/pgoutput/pgoutput.c
+++ b/src/backend/replication/pgoutput/pgoutput.c
@@ -60,6 +60,9 @@ static void pgoutput_message(LogicalDecodingContext *ctx,
 							 Size sz, const char *message);
 static bool pgoutput_origin_filter(LogicalDecodingContext *ctx,
 								   RepOriginId origin_id);
+static bool pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+								   ReorderBufferChangeType change_type, bool in_txn,
+								   bool *cache_valid);
 static void pgoutput_begin_prepare_txn(LogicalDecodingContext *ctx,
 									   ReorderBufferTXN *txn);
 static void pgoutput_prepare_txn(LogicalDecodingContext *ctx,
@@ -271,6 +274,7 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
 	cb->commit_prepared_cb = pgoutput_commit_prepared_txn;
 	cb->rollback_prepared_cb = pgoutput_rollback_prepared_txn;
 	cb->filter_by_origin_cb = pgoutput_origin_filter;
+	cb->filter_change_cb = pgoutput_filter_change;
 	cb->shutdown_cb = pgoutput_shutdown;
 
 	/* transaction streaming */
@@ -1749,6 +1753,72 @@ pgoutput_origin_filter(LogicalDecodingContext *ctx,
 }
 
 /*
+ * Determine whether a change to the specified relation should be published.
+ *
+ * See the comments atop of LogicalDecodeFilterChangeCB for details.
+ */
+static bool
+pgoutput_filter_change(LogicalDecodingContext *ctx, Oid relid,
+					   ReorderBufferChangeType change_type, bool in_txn, bool *cache_valid)
+{
+	PGOutputData *data = (PGOutputData *) ctx->output_plugin_private;
+	RelationSyncEntry *entry;
+
+	Assert(RelationSyncCache != NULL);
+
+	if (in_txn)
+	{
+		Relation	relation;
+
+		relation = RelationIdGetRelation(relid);
+		entry = get_rel_sync_entry(data, relation);
+		*cache_valid = true;
+	}
+	else
+	{
+		entry = (RelationSyncEntry *) hash_search(RelationSyncCache,
+												  &relid,
+												  HASH_FIND, cache_valid);
+		if (!*cache_valid)
+			return false;
+	}
+
+	/*
+	 * If the pubaction is not supported by this publication then return true to say
+	 * the change for this entry can be skipped.
+	 */
+	switch (change_type)
+	{
+		case REORDER_BUFFER_CHANGE_INSERT:
+			if (!entry->pubactions.pubinsert)
+			{
+				elog(DEBUG1, "Filtering INSERT");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_UPDATE:
+			if (!entry->pubactions.pubupdate)
+			{
+				elog(DEBUG1, "Filtering UPDATE");
+				return true;
+			}
+			break;
+		case REORDER_BUFFER_CHANGE_DELETE:
+			if (!entry->pubactions.pubdelete)
+			{
+				elog(DEBUG1, "Filtering DELETE");
+				return true;
+			}
+			break;
+		default:
+			/* allow any other changes that are not explicitly filtered */
+			return false;
+	}
+
+	return false;
+}
+
+/*
  * Shutdown the output plugin.
  *
  * Note, we don't need to clean the data->context, data->cachectx, and
diff --git a/src/include/replication/output_plugin.h b/src/include/replication/output_plugin.h
index 8d4d5b71..f82bc1d 100644
--- a/src/include/replication/output_plugin.h
+++ b/src/include/replication/output_plugin.h
@@ -97,6 +97,25 @@ typedef bool (*LogicalDecodeFilterByOriginCB) (struct LogicalDecodingContext *ct
 											   RepOriginId origin_id);
 
 /*
+ * This callback is called before a change record is decoded to determine
+ * whether the change can be filtered out before entering the reorder buffer.
+ *
+ * Typically, the output plugin needs to refer to the system catalog to make
+ * this decision. To enhance efficiency, the plugin should cache the lookup
+ * result for each relation, minimizing catalog access on subsequent calls.
+ *
+ * If the callback is called with 'in_txn' set as false (the reorder
+ * buffer has not started a transaction), the output plugin should set '*cache_valid'
+ * to false to indicate that the result is not available in its internal cache.
+ * If 'in_txn' is true, the plugin can create a cache entry after querying the
+ * catalog.
+ */
+typedef bool (*LogicalDecodeFilterChangeCB) (struct LogicalDecodingContext *ctx,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
+/*
  * Called to shutdown an output plugin.
  */
 typedef void (*LogicalDecodeShutdownCB) (struct LogicalDecodingContext *ctx);
@@ -222,6 +241,7 @@ typedef struct OutputPluginCallbacks
 	LogicalDecodeCommitCB commit_cb;
 	LogicalDecodeMessageCB message_cb;
 	LogicalDecodeFilterByOriginCB filter_by_origin_cb;
+	LogicalDecodeFilterChangeCB filter_change_cb;
 	LogicalDecodeShutdownCB shutdown_cb;
 
 	/* streaming of changes at prepare time */
diff --git a/src/include/replication/reorderbuffer.h b/src/include/replication/reorderbuffer.h
index 363d9cd..0131022 100644
--- a/src/include/replication/reorderbuffer.h
+++ b/src/include/replication/reorderbuffer.h
@@ -494,6 +494,12 @@ typedef void (*ReorderBufferMessageCB) (ReorderBuffer *rb,
 										const char *prefix, Size sz,
 										const char *message);
 
+/* filter change callback signature */
+typedef bool (*ReorderBufferFilterChangeCB) (ReorderBuffer *rb,
+											 Oid relid,
+											 ReorderBufferChangeType change_type,
+											 bool in_txn, bool *cache_valid);
+
 /* begin prepare callback signature */
 typedef void (*ReorderBufferBeginPrepareCB) (ReorderBuffer *rb,
 											 ReorderBufferTXN *txn);
@@ -604,6 +610,7 @@ struct ReorderBuffer
 	 */
 	ReorderBufferBeginCB begin;
 	ReorderBufferApplyChangeCB apply_change;
+	ReorderBufferFilterChangeCB filter_change;
 	ReorderBufferApplyTruncateCB apply_truncate;
 	ReorderBufferCommitCB commit;
 	ReorderBufferMessageCB message;
@@ -779,6 +786,9 @@ extern uint32 ReorderBufferGetInvalidations(ReorderBuffer *rb,
 extern void StartupReorderBuffer(void);
 
 extern bool ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
-												XLogRecPtr lsn, RelFileLocator *rlocator);
+												XLogRecPtr lsn, RelFileLocator *rlocator,
+												ReorderBufferChangeType change_type);
+
+extern bool ReorderBufferCanFilterChanges(ReorderBuffer *rb);
 
 #endif
diff --git a/src/test/subscription/t/013_partition.pl b/src/test/subscription/t/013_partition.pl
index 61b0cb4..a64b27e 100644
--- a/src/test/subscription/t/013_partition.pl
+++ b/src/test/subscription/t/013_partition.pl
@@ -472,6 +472,13 @@ $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4 (a int PRIMARY KEY)");
 $node_subscriber2->safe_psql('postgres',
 	"CREATE TABLE tab4_1 (a int PRIMARY KEY)");
+
+# Ensure that the subscription 'sub2' catches up with the latest changes. This
+# is necessary for the walsender to update its historic snapshot. Otherwise,
+# the walsender might retain an outdated snapshot, potentially preventing it
+# from accessing the newly created publication.
+$node_publisher->wait_for_catchup('sub2');
+
 # Since we specified publish_via_partition_root in pub_all and
 # pub_lower_level, all partition tables use their root tables' identity and
 # schema. We set the list of publications so that the FOR ALL TABLES
-- 
1.8.3.1

v18-0003-Tests-for-filtering-unpublished-changes.patchapplication/octet-stream; name=v18-0003-Tests-for-filtering-unpublished-changes.patchDownload
From 7a26e5a3a55b0bf32406dc314d208584a20430f9 Mon Sep 17 00:00:00 2001
From: Ajin Cherian <itsajin@gmail.com>
Date: Wed, 30 Apr 2025 06:52:30 -0400
Subject: [PATCH v18 3/3] Tests for filtering unpublished changes

Since filtering is throttled, this patch removes throttling to be able to correctly test filtering.
---
 src/backend/replication/logical/reorderbuffer.c |  6 +-
 src/test/subscription/t/001_rep_changes.pl      | 89 +++++++++++++++++++++++++
 2 files changed, 94 insertions(+), 1 deletion(-)

diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index cd2b052..bd12d77 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -378,7 +378,7 @@ static void ReorderBufferMemoryResetcallback(void *arg);
  * hash table search for each record, especially when most changes are not
  * filterable.
  */
-#define CHANGES_THRESHOLD_FOR_FILTER 100
+#define CHANGES_THRESHOLD_FOR_FILTER 0
 
 /*
  * Allocate a new ReorderBuffer and clean out any old serialized state from
@@ -5800,6 +5800,10 @@ ReorderBufferFilterByRelFileLocator(ReorderBuffer *rb, TransactionId xid,
 		entry->filterable = false;
 		rb->try_to_filter_change = rb->filter_change(rb, entry->relid, change_type,
 												  true, &cache_valid);
+		if (rb->try_to_filter_change)
+			elog(DEBUG1,"Filtering change for relation \"%s\"",
+						RelationGetRelationName(relation));
+
 		RelationClose(relation);
 	}
 	else
diff --git a/src/test/subscription/t/001_rep_changes.pl b/src/test/subscription/t/001_rep_changes.pl
index 916fdb4..1de1c0c 100644
--- a/src/test/subscription/t/001_rep_changes.pl
+++ b/src/test/subscription/t/001_rep_changes.pl
@@ -571,6 +571,95 @@ $result = $node_subscriber->safe_psql('postgres',
 	"SELECT count(*) FROM pg_replication_origin");
 is($result, qq(0), 'check replication origin was dropped on subscriber');
 
+$node_publisher->stop('fast');
+
+# Additional tests for filtering of unpublished changes
+# Bump up log verbosity to DEBUG1 for confirmation logs
+$node_publisher->append_conf('postgresql.conf', "log_min_messages = debug1");
+$node_publisher->start;
+
+# Create new tables on publisher and subscriber
+$node_publisher->safe_psql('postgres',
+							"CREATE TABLE pub_table (id int primary key, data text);
+							 CREATE TABLE unpub_table (id int primary key, data text);
+							 CREATE TABLE insert_only_table (id int primary key, data text);
+							 CREATE TABLE delete_only_table (id int primary key, data text);");
+
+$node_subscriber->safe_psql('postgres',
+							"CREATE TABLE pub_table (id int primary key, data text);
+							 CREATE TABLE unpub_table (id int primary key, data text);
+							 CREATE TABLE insert_only_table (id int primary key, data text);
+							 CREATE TABLE delete_only_table (id int primary key, data text);");
+
+# Setup logical replication publications
+$node_publisher->safe_psql('postgres',
+	"CREATE PUBLICATION pub_all FOR TABLE pub_table;
+	 CREATE PUBLICATION pub_insert_only FOR TABLE insert_only_table WITH (publish = insert);
+	 CREATE PUBLICATION pub_delete_only FOR TABLE delete_only_table WITH (publish = delete);");
+
+# Setup logical replication subscriptions
+$node_subscriber->safe_psql('postgres',
+"CREATE SUBSCRIPTION sub_all CONNECTION '$publisher_connstr' PUBLICATION pub_all;
+ CREATE SUBSCRIPTION sub_insert_only CONNECTION '$publisher_connstr' PUBLICATION pub_insert_only;
+ CREATE SUBSCRIPTION sub_delete_only CONNECTION '$publisher_connstr' PUBLICATION pub_delete_only;");
+
+# Wait for initial sync
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_all');
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_insert_only');
+$node_subscriber->wait_for_subscription_sync($node_publisher, 'sub_delete_only');
+
+# Insert into an unpublished table (should not be replicated)
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO unpub_table VALUES (1, 'unpublished')");
+$node_publisher->wait_for_catchup('sub_all');
+
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+ok($logfile =~ qr/Filtering change for relation "unpub_table"/,
+	 'change for relation unpub_table is filtered');
+
+# Update, delete and insert tests for publication with restricted tables
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO insert_only_table VALUES (1, 'to be inserted')");
+$node_publisher->safe_psql('postgres', "UPDATE insert_only_table SET data = 'updated' WHERE id = 1");
+$node_publisher->wait_for_catchup('sub_all');
+
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering UPDATE/,
+	'unpublished UPDATE is filtered');
+ok($logfile =~ qr/Filtering change for relation "insert_only_table"/,
+	'change for relation insert_only_table is filtered');
+
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "DELETE FROM insert_only_table WHERE id = 1");
+$node_publisher->wait_for_catchup('sub_all');
+
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering DELETE/,
+	'unpublished DELETE is filtered');
+
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "INSERT INTO delete_only_table VALUES (1, 'to be deleted')");
+$node_publisher->wait_for_catchup('sub_all');
+
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering INSERT/,
+	'unpublished INSERT is filtered');
+ok($logfile =~ qr/Filtering change for relation "delete_only_table"/,
+	'change for relation delete_only_table is filtered');
+
+#cleanup
+$node_subscriber->safe_psql('postgres',
+					"DROP SUBSCRIPTION sub_all;
+					 DROP SUBSCRIPTION sub_insert_only;
+					 DROP SUBSCRIPTION sub_delete_only;");
+
+$node_publisher->safe_psql('postgres',
+					"DROP PUBLICATION pub_all;
+					 DROP PUBLICATION pub_insert_only;
+					 DROP PUBLICATION pub_delete_only;");
+
 $node_subscriber->stop('fast');
 $node_publisher->stop('fast');
 
-- 
1.8.3.1

#49Peter Smith
smithpb2250@gmail.com
In reply to: Ajin Cherian (#48)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hi Ajin.

One comment for the patch set v18-0003..

======

1.
+$log_location = -s $node_publisher->logfile;
+$node_publisher->safe_psql('postgres', "DELETE FROM insert_only_table
WHERE id = 1");
+$node_publisher->wait_for_catchup('sub_all');
+
+$logfile = slurp_file($node_publisher->logfile, $log_location);
+ok($logfile =~ qr/Filtering DELETE/,
+ 'unpublished DELETE is filtered');
+

For this delete filter test, shouldn't there be another log file check
missing here which names the filtered relation? -- e.g. something like

+ok($logfile =~ qr/Filtering change for relation "insert_only_table"/,
+ 'delete for relation insert_only_table is filtered');

======
Kind Regards,
Peter Smith.
Fujitsu Australia

#50Ajin Cherian
itsajin@gmail.com
In reply to: Peter Smith (#49)
2 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

Hello,

I ran a series of tests using both streaming and non-streaming logical
replication modes with the patch. In non-streaming mode, the patch
showed a significant performance improvement — up to +68% in the best
case, with a -6% regression in the worst case.

In contrast, results in streaming mode were more modest. With the
default logical_decoding_work_mem of 64MB, I observed a +11.6%
improvement at best and a -6.7% degradation at worst. Increasing the
work memory provided some incremental improvements:

At 128MB: +14.43% (best), -0.65% (worst)

At 256MB: +12.55% (best), -0.03% (worst)

At 512MB: +16.98% (best), -2.48% (worst)

It's worth noting that streaming mode is enabled by default in logical
decoding, and as such, it's likely the mode most users and
applications are operating in. Non-streaming mode is typically only
used in more specialized setups or older deployments. Given this, the
broader benefit of the patch - especially considering its complexity,
may depend on how widely non-streaming mode is used in practice.

I'm sharing these findings in case others are interested in evaluating
the patch further. I believe the worst-case performance degradation
can be reduced with better code optimization. Feedback is welcome if
people believe it’s worthwhile to continue development based on these
results.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

increased-work-mem.JPGimage/jpeg; name=increased-work-mem.JPGDownload
����JFIF``���ExifMM*;J�iX����>�Cherian, Ajin������37��37���2025:05:28 13:41:242025:05:28 13:41:24Cherian, Ajin�� http://ns.adobe.com/xap/1.0/<?xpacket begin='���' id='W5M0MpCehiHzreSzNTczkc9d'?>
<x:xmpmeta xmlns:x="adobe:ns:meta/"><rdf:RDF xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:dc="http://purl.org/dc/elements/1.1/"/><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:xmp="http://ns.adobe.com/xap/1.0/"><xmp:CreateDate>2025-05-28T13:41:24.368</xmp:CreateDate></rdf:Description><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:dc="http://purl.org/dc/elements/1.1/"><dc:creator><rdf:Seq xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"><rdf:li>Cherian, Ajin</rdf:li></rdf:Seq>
			</dc:creator></rdf:Description></rdf:RDF></x:xmpmeta>
                                                                                                    
                                                                                                    
                                                      <?xpacket end='w'?>��C

		
'!%"."%()+,+ /3/*2'*+*��C
	
***************************************************���
"��	
���}!1AQa"q2���#B��R��$3br�	
%&'()*456789:CDEFGHIJSTUVWXYZcdefghijstuvwxyz���������������������������������������������������������������������������	
���w!1AQaq"2�B����	#3R�br�
$4�%�&'()*56789:CDEFGHIJSTUVWXYZcdefghijstuvwxyz��������������������������������������������������������������������������?�F�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(������5�W�����/����UHH��8�J���QK^Ko����i����F���%�c��m���?�\�g��gT��
x�8�X��a,`(�)����(�  ����B@�@Fk������	�{�%����Kx�>^���<g
q��^�o}��~��x�Y��U�o$���2�6��g��G��8�=(��L����3I�@E���QI�~4�QFh��"��
(�  ����)2?Z(��
3X�+�5��|9s��$��H��J����'��������$��b�_>������<��c���O=q��\�^Qw��>��xw������"��I<�"�>�i��������H�����G�h���QFi23���Q��QI�:��Fh�R��-Q���ZM��W�vS��n��($lb�#�=����B��9�+m7Z��w9{������5��S�Z(���L�Z(�4����Z(�9�����~3k:����k��[;���D�����;��QK^A��?�����p|E�V��I�) 
�	�j�������]�'���
Z��/��|��QI���4Q\�����
�T����i��<~b�\�������u��Z����][��l\.O��
�(���}(h��H�x;X�'h�,�etb
���GC\��[�����3���s+I.d�Fv?7rI4�P=(��L�Z(��k������V[YHe8 �=����.�����S�K����<�#c#��&��
(�QFi	��Q�(��_�������X��.d��� c�<nS�3G�=F�V�o��jw2]]M&���a�=�uTW)�7Q��~k7�e���P��9���;�dg���j���}�R����hI4���q8���3@ ���)2?JZ(�4QU�[N������5����/%�����j(y�i
��X�(�����x����}J�<����kppe��{I���4f�J��������x�O��
���E�*�FFF8�$����=�/�2�w������]|�;��!\�W�����=Z�@x�#����(��|m3��vX$x�M>vG���!#��O�wW�4��.%���m�M!v?�a�$��h�4J(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��+�h��,�����Z�%��?�C�������:�>��m���n��"���#u�#�������������rH1����!S'2+���H�VS�
iFV�6.m�J�����i������![�h�T� `���=�U����j���w��n�s����[K��c�#$��Jo��=��<K�Mr���)�/t��������Fs���Q����,�/-�{�J�V��1�mou��v���]��<&bo:�Y1�-��`|�h�S������GN�X���,RF�+2��9�<#���ON��_���n�*��Y����p���lg�j�~*XC�����fh-`���$������]����z?��/��<��'�����m�8u}F�l��b��m������'_s^��������Y�������hn�F��I��+'����o��b�����w�����Ki�BS��<E���5k�����^i��������cHz�zg$����I�}W�7=����b��Fi�+��w�zs��@ :����{^�t9�w}ez�$M�m ���w���oo���=�M���k_�B����R(_���g�5�j:t�F&��X���0
"��pk��w��xCM�|]�
M-|�K+;9��$v9�����[�;��N�����f���7�`��~u�|7�i���>/��<���z��&�>��^\�%�a�&a��(��������1.�c%����^}}����aY=����8����~�'��
+�h��s���?�6��\���I��.m�m��v�n�@�����:=���h����_��W�"����o3=����2������/����I�C�����P���L��u�hV��e��*��2�|��=��k����Z�n�������S@Cz�<��������
WK�����m��
!���3���H�#�x��e���R��W{��\o$d������?��/��k�������%��}�H���z�~2Gj���}�������h����<��ngq��6O>�K�4zD?�G��c���x���`B�$I �O�[>6����_�{��%K������"�V�6���R��@3�G*	�<V���^����.�!���v��V��`�}��?J���K��y7��3�=dt�����y���/��S���k�+�:L�S�V��
�:���v$�Ei�j�G�-�t�v�C*���bJ��������>�=#�)���j	xeX���	���
{1`�,p=My��OjZ����m�u�jo&N?r��=�pnOj�������v���jmU�Z�.���B�q��7�=2�����g�=&F6�B�Z?�s����5�G�'����G��N�+-�C����$�a�=]u�g��%��u��CAxz�8���A�^pzW�W��Z�u�o����.����sp�$s��u���<[����_
�{�Csv�@_����k�-������H���4��	��
^]"5�������B��k��w�,�]��]N�Uftx����2����s@��|V����{\��M�������� ����������L���P����%,���UG���|}Cy�
W�[X/VF ��q������U�R��^%��Y�A#�78XX�����~k������x�������Y�pI���<�"�?M�-�].���A-��bH�Nr�y���~�R\��K�f��"r��#?�>?�&� ��w������������=��$,\
gh��vk��[N�������[�[���M��_,�F���s��MK�?�9�
���RW����E-[��f��m@[�;}s�����{�?J�Qo�������>i]���
��sQYZ�|,�����������
��7��8��#k���$��������7�����������c�?�n,|c5����/�sC+!#keH8����[w��.-_�~!�F�u� ������@�9��zs��'�v����58���#��������H'�GFS�����y���kY������#�}A36�u!%��8��s��T��W�O����$��%�����ZI��X�����:W����QF�y�b���\H���X�����3�V����#�z�����V�U���y��1����W�>	�<+��k>	���Z������L��������~�x��:n�*�Kq�UzS����k��w�����<�j#IX�RKWk��a07`���v5�|��G���I��{���;�<A�MG_��^�����i.��f3Hw2�J����I��zX�]��|	�e���	�������{��+1\��>�,��������i�"6��^��9_�7������5��'�H]�H9=�\W�|2>+�]��+�6��&��X���
0G��<`v�k�� ��	t�v
��I�y���xgU��R������;��|�T��v�����������:��we,>_��e�6A��n����X��c�.?�{P�M��/7Q����`29��-�JO��"�����}_Y}r�ks,3�3$_.���p ���T:4����Z���j��;�����a����	.|5���x3�z�z��fb���Y����q���v�
�\�4�U��:�])0�b� v`�5���N�G���}"�6�Ic�� �$��?���ko�oq2�[��$��\�`�T�|/�5�X��u��g��w�w�f3��`F����r+C�Z
���WJ�|7�j|�"�K����`A��8�����E�s����E�M�B��c�a���G���q������j����Q���N����|.��M���/=��_jr��e�����T�����!�+]'��b�(���U�I����oL�����}o}n���@�c##�"�/���|��Y?�}���W�W��|��Y?�}���P��J�~���*�Y�,1$���V�k��d�x�����t�ySH����������}NKc��w>��.|#��O��.I����G�B��$�rs]x�
/Mu����B�Hx�y�����R�n�x�T�3<0-��(O3��W)<��g�4��?��5�P��VS\��H�>����������0�&�����]�klf*��@�+3����#�-GB����Z��r$�,��F��q����=���R�=����P��+Rb9�2��w�j����
c��N��%�-�����X���v��[�$�@��u?���������@*r>��#���~)x�Y��4��J_#U��^�u�2H=�1'�Z���
l$��6�6�FZ�.����H'�f��d�����sQ>]��y>sp��t9=��)�^��A ��@���s�|)�4��BmJk}>�M��~A�#<����W�|-�v��_[�������Z�?�����s�'c�����7�u����?��\����%6��_��9Qe�����ce�^^�Z��-���6B�2)���|T����<3�R��5	�������lw�or+����#����	�X*��	bd��5	�Ceov����
��9�~R�\��3����G����ZA�Ia�
���s���*_�������P��5��~���Hr�)?C�=�E��_��ug�k3A2��D��e=�kc�_	�;�OC�i�Z������st#�������R�|o�-|�ji�6�	��	���p(�nzQ�|5��W��5k����n����o6+���� d��G����1����m��f�y!�|0���o���?��{��T�m$Q��������r���:}�o�+�k�����3�?�1^�@E����|a�|C������]�ec����8=?���;Y�`�4;�N��
�-3s��3���s?|7�	7��B�u�����w'$~+�~5�^#���-�K���M���:�\y��$�J�h��4����|U�Kqsq�O$�E$��"�p���==s>���-��u���P7��,,�b����x�����G�-��N��Ag�	���������^�/����.�����.�6�by������/�K���?Z���^������l�%��A����������/	|Z��:��&�cs�f�������A� �W���w�I>��\�CZ�<�6�C�6�����j�-�*.m��������W�w�IF��\W�CZ_��J4/����1���{�6�]�.�����kOK�����8�`r,W{��x����6���gv=�f$��/�&����������P�ge���.h������WB��~2\]�Z��y�h����YK������2H<��)�#�o�_�k[���D���{7��=�oQ�q����S����1����W�������]y{W����>��q7�_	[���|�2�������Vs����������h��@2�����G�y���
����j�_��iu47��������S����[
;D���'�;&���,A��h��Vt����O��l��d(�f�q����H�
�	|K���k~�u%�d�X5��m�lD����c�Gj��X�|Q�
����o�N����7Q.3��pq����~���/G��T�#�)l�
�E�H$���J��Cp��G������W?X�}q@���^�'�����mI-���dWj��p0H�:������v��j�|�z�u;m#�����[�1c�� �8��������F������z�x������k��-��h���z�a�D�{mx��=:�O�<?����s������>��?�B�=�t���=r��Z<��:�m:�R*{��#�&�����Mw�]Gk������>��8o��2�<9����;um^o"X��H�$�nOj���rK���k-�2�k��>Z���H������]��~�eku�V��+s�����ku�'W��2���S� �b�<��G����M_��*���z;|����89=���
�J��
���+����t�Jm3E�S7�[���������8'�O�����������,b��'�v����c��xT��:��J�"9�X�o��0n���l��"�����xCR�|)��7��������Iba��xe�?�u�Ko��l�6������W$���R�K�g�������o�����^��X?�d��

�?�'��4���s�-������X����/������YK����&?�� �x��%���-���/�^�i�:G{����V8�==�d<������v�u�{�[��W
���^XU�n|�}EwS|������g�-��`^V�=2��+�.�����dC��2?CSUm6�-7K���,��B��X��Ud�8f�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(�4QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEW/��[x�E�N����"�LRI�|����(Kqie
��aj���������O��J����i�����/u`�w�R=}�3Yg����	�Ok���<��mS��5�P�<-o�	O�K<��L��d@P����{C��C�l���yR��]���d���Es^;�]���>�U������8�R��F>n?��![�$��('�*J(��7�m;[�����W����qb�����j-������^��W��a{���9�;d�����g�b�_��4K�����(ib���q�;U�H�@�����4vp,+#���d�����9ox��Z���\^�n�L�ti��������(�[����D�V������1�Tm8�JO���v6V�w��-���V�T�;H��:s]U��O
��+���ux���`�Nte=������[�>k�i�b�lR����}z��xk����O�k�T�����l���������<������<y�x�[���KB�*�~[�#?����(w!_Q��<�{o��vv��������ePT���WQEq>1�]�x�P�TY�t�V0��m�q�p��^
fY|%�:�����]sV��U�-���,�#9'��I��3��N�&�6��[-���T�T�`{�+���7{�����w�i�9�m�L��*�*(��g��?�ut����{xf���q ��k���l����H����V�W����WS{�
���&����O��=����(����1�����Fr ���~<�����O�4W�/-R["�-�h�����Q@V�$�Ym�/kzn�)%�#�+�� ��"�xG�>���Dn��A��c�IH�=��������
m�O�c�����b[uU�����g����l�Q���������P�FG�Vh�{���>�G����4���&UK��*��<
m�OhZ����.�/�Q����V�dg�{WSEy?�	�"N��a(���S�!
�O
x�V�-n�y��d>I$rTdc>��x��_���������a2��1���}kj4���(��O�t�XK�$����qw9�������Q���-;�����������yhp���n�`�mEy�? �Y���"�5���Hm���-�u+�T�������������-�e��������r�����Er��%�����{=������U<����_NM_F�����K�u*c#?Z�Es~�]��|����slRTi$
�C�=;�!���9|��z��k�� ���:(��o�M/���?h���[�VK����=@�������_��?�-����������;d��=�y�������R��XS��/���([)k4�_�'�+�,����(m,�H-�@���
�=�x�����--/'�[��p�2�*v���t����[x�F�N����!�Y���I ���5�Q@�t�����FT���.�v��X��e��|>�M���q��?�:�l�/�k���
��o�<m���������I<�B�Nq���@��94��N�F�- HU��PN>�p��Ey����b��5
��S���b��h�)$��3��K���=��.��K~�Ks>�1Y%���K�W�Q@���mSM���!5��f)Q��H��6�������<W�X[��E8�9�W�Q@�'�������n���;���x���{�b�7�[B,���A���l���\~��tP
���t����\so$t�O�$�=�X���
�����Z��u,�$��q�n+z���Y�ko�Z�sy=���|i���l�� ��[���e�������\���d��?�z�Eyj|�MgO��\�-��h�����+�-|+�����j:��4F��]�) �>�WgEr~7�y���������Q{n@�9�=�y�=+���K2���M���
����L�@m�������TP;��A��
"��k���/�0P�n�q]P2�Rd���9���}��2]~���c�I
���\e��3�x�G��!�����kY���F�88#���E�j�N/��^����.@��tP���z_��	�F���5i�W��m����8����(#�>��>���'��;��Fe�������^����o<��!E�@6X�q�z���2<S���S����y-��@�,@\0<g����"���lt[y��;8�,��'�q������	t�g[mkI���uG9��������_q��p�5-���	/���^9�m�~�&�>���'���x~MV����T��F��)�Ep���Ku���m�M�
���J�w��
�J(����m3�~,�5�.iQ/ -����o�=O���j�������(�/2�rM��E���p8>��Q@ie�y^��oj�����6�R)[�#'8�����
��6�6�iw5�Kp���TH(��t�PP��A}i-��)<)I#�r���"���<�o�v�w�\�K�z����-�����8��SY|��R���:���a`����  �pO�^�Ef����mm/X��L9�R:2��z����s�j�}����t����o�\~���P'�|%�x;F]7E����H�s���c����<m��J���^�4�v�l������@#�;w�Q@i��n�f�7�5]v!��k���R7��r�<g��<��|!�{]��[����,�;��8��m�@��iz��u�2��B�\�K���R=}�3Y�����*������n0<�T�rMz���G���
uPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEP^k��
-�.N?������W�W�i��s���q��@�ECup-��V�����5&@��f9z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z���R����h��r��4z�!�lO`j��K����������o���G���
�@@'8���MY��DF�c�$b�����]��H�?v�4(�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@�%F���eE������{!�8�E|����OZ�@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@W2��i�b-��u��X��C����\��c�����(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
�M;�NcV��n/�+���4��9�[�����p��W�����:�qq���
�(P��3�$�U���?�/���E�"���^�����I2d����O�5�A/��'�G�'�����@�������������x�U�/���h����_����4����	�	?���z(�������U�/���h����_����5��k:��\����+��l+�r���W�[�"���~7������R�K�-���:p����r�#���kv��/5}@H��q���sc�dV�q������������]>+h��w�4�kI��eX���Lp��>`N3�]�`@ ���z�������^��%�$�������$PF�i2�I��{�N�m��~3�����$K
��\y���7����MwX���FI��z>�c��0jzL�}��-�
���G"���Z����hwZ��2���Qd6�F']��`T���j/���xJ�������lM�6�������z�a�>&�G���=�����F�+"���A��j��1��~<���.��}WN�yi-,��DQ����0w����x��Q&����E��4F�X��>R��oB���L���r(�91j~����W_��]?[3A4:��C�����c��J��|e��M[N���gG���V�8�{IY��nY�TF��/���^��Xi�7l��<�A�m��~��~�������=���e�[}�������U�I��(�1��B�����7)��V��{�-��{Y	1����y�@����#����E@��,�'`J:*���]��e����������ogl�I&�g���GR:�/���[��t�x���`��8�}�����t��3i�%���
y�ena�g��<�5[W��e���m4�cQ��$�Aai��V�O�+N��O��k�R�H�-���h�u3��C�W��~�sR���S����<9�iv�wk
���Rvd�R1���u:���`�CN�4����B�����2x����}K�&�a���ZZ�Z��������8��3}�MU���5
OO����8�5�2���h��GL���������]�9�����6�n�R�p��_�X�fp� '��4�:����+����1�}�"�A	���95{@�l�I�Z��ismr��H�Yy�v ��o��|]�=�j���}�49��"���u��
x�H����G��������%����w���jZF�.Zm8�,��g�s� ����J�k����^���o,��da���j��h��������-k������|���bj��5���H�/�k������#�k|�.������=3�~$�|S���"s,A�r+�G��Ue<�*��� �����'�6���]9�Y���}F��S��������]��� �������- �K@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@]����g>b����J���$�yGJ���3�X��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��-/������x�vF?L��\���
����A����,QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEW�i��s���q��^�^i���j��-����w���x������^hw����o$h	�IR4���x�����qn-4&��x��+����?�������C��^�Ez����/��y���{���x��+����?�������C��^�E���/����{���x��+����?�����t=
kmJ5�V�iW
��Q�}
t�V��k��IX���sF�\��"�^��izdk%����$g
	��x��\aOI���E���B�
�q�8!@<�.���x�Y�t
2�W����o:�����v�89^��4P����f����4�������S�`���<�f��}3���m�)�����p	c�W��BX�b#99�Z���9�t�x"��I��u�2o��Iy�w�2I	"�s���S[���S���^	���K��Tf��<���z���S�����	_���o��M%���EF~}+:�C�'�<U�k���,�vYn�'��9���|�:�]���Zk�%��[�������"��s3�����g��W5��?	I�F���Gl����^ �E��#$�u�M��0<
�������F5����#��`9����2��z��m�x^�Iip��E��`��1��c�w9?�]���V��6Mg����t}*�$�SI|� PNO|�����[=R�=7��~�dH�<�o�\�{�WYEq�����O�j�%��mWZ�^x��b�5]�'����.�����Z}����
?R���{�����Wbv�����zu�J��<�U�w}i�No
�L����I#��I�����J�u����j���z[n���	�#p� ����(���
F�����K��x"���S�z��s�������m�|-o
��5�s6��|��\L�5�4��x�:���k:����jZ���oj���p�����k�����U�um1�c�����9������KH:R�EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPk�G.��P�����5Z�F����Q������EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPE6YR�I�c��v8
=I�_��G�l��%?��FR�	�-�~������V��S�h���?�+e��)�4���2y���EP���?�+e��)�4oi������=��0��r�C�{H������������V��S�h�s���=��U��#���_���G����A[/�	O�����x�/�T?����
��J�'����A[/�	O�����x�<��?�������c$v�RB���`�@'�g���?����r�p�4�&���$N��1VS��y�������8E�t>Z�;�������5��i�����G�/�s����'���QZg�������_�_���4���O������@�?����k����?����O����\��F��}��4���?����r�yUg��>���c��~k������?���_���4���O��*����/���?�z��/�s����'��i���w�7��������B�|��v��s�������>��mw�5����#
�8\���
��0�z8iN1I�~g^Z�u�O���U?����������4ki���k���k���?�_y�\��[��ki���k���h���������/�����2���=�tUO�m;��������w�������=�?�_xsG�n����������_���[N���_����G�����h�-�U?����������4ki���k���h����}����*���{0U��$�&^O�V��8��wi�QEP��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���N������[��G
���M;�NcV��n/�(����������M<V��Oq"Ej]�v���d�OAUu��?�/��~2��]��u���j�G�J=���b�����c��-+�c�?�3���ZW���5�]�������?��S�Q���&~���J�������L�1�C������|�E�������O�G�����c��-+�c����;
V������B���o(�Cc8���������7������E�\x��j^�J�^0�"�#��M����g�iw��8���M������V�����$��?������l>'x;R����]����������������U�}�
!���os�k3APG����m��e��?�t�"���u�����W�b[h�	�I%�@�f&F�O��@f
���Oj����Q���O����A�����85��W�:��<7�i������?����K$S#.�p���0#�Q�Gc�[xH�'�	:BA#7�Q�����x���P��^��-+X����3�������Wq���q���<q��o������\3Qd����@i��p1�A�����o>�>�������1(g�]��������O������k����c�	c�E�X�T]���R3����|{%��u
���G��t�X���#�q)!�W g��O�n�c����l�p��>���?����?���&���j��������<����B�#i��G��������d��Y��E����	s�xS�r3�z&���k���a����,�y��������<S�xZ������y<��Fm���
aY�8�[��9`��^*��
#Z�n�9u�YGy6������q�����G@hOB���<I~�z.�.n�<�
8�Y@�F����b�����ki8P�%�]�	�f��4����/��EN�sKi$7hnB����CB*���6����������,o�en�@�w0�j�)"�]��i�>�����k�j�"�.��8c����co�[Z<k/��c�C#ZFOB

�<.��|=�����F���(�f��Q@��x���u}oN���5=Z�m����'�'��4�R�Y� �4�����M�J������Zu���7�����-�-�\���_�>�?
�N�W�:W�4��i,ot�?��(��JJ���K�n��k�^��Gsybqs+~��oR0y��Y7?<kw5���,�H���	N�d&+��g�7��$j:|�7+��Ir����ZB�D�~�W�~%�>��=�����^tc���������=_J�,5�:+�&�+�Y�,O�o��&�� �������q������������g�y��[
���Wq��9������KH:R�EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPk��?5Y����q�x�5Z��#��+�T7bOZ�@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@^)��OU��I�^��^�����_����A5��yG���x���A0=(���������JZ(0=(�������JZ(0=(�����R�����uo�Uj���!k��������c��QE2B�(��(��(��o�������rq�s�q5�|>S�_����6��<���G��+|���t������J(��P0=(����0=(����0=(����0=(����>��?�������n��c?�<|w��������{�����^\�t}+��qh�\��:hNQN����������0����������0����W���G�~����z��/y��^���G�/y��^���^KE_�7������_�����z?�?�E�����z?�?�Ey-}���k>����������0����������0����Q��G�~=�������������R���������0������_�7������?�����l�V�[��������p3������e�$�F���Cj���iI��[�����0��+B��(��(��(��(��(��(����F�/�_B�V�}V5,�k02��y^�5����'��=�_i����u��<���z��W�xc�-�_��-���u�\h��j,rg�v%���{
�+��M6�K�B�+[x�^Y�"�������B������j>_������^u�hc���xORv�ee&�sl~��d�;���j��E���x���pi�6��V�+k�n �������=z��Kw��X��1������ROJ���_��\<&�a4z;{�v�5��S
�Ox;��
;P�K����&�!����U�0�cL����|Y��>�L��+�d�KH�[�,���r:����I"EI+*"���p��m3���u����:��s�`�Vn=�����.���K����0x��4�(pZ����|U���|6���;�[i����ueqi����$����Id�
�����/�o��?i��go���Fr}1�k�~%x��P�1���������pUd���CZ�0�[����X�ZU����h�Z]�Y�X�pm��I�h���4��9�[�����p�����?��hz��o���W���(��*MrK������6�{ogxvW��cR|��#�(����?�/���*��������m,������%��ri����i��m���������z��y����"�2���bd��G�_��x����G��?��g�����5�nW����{�U�G�M/�g�_��x����G��?��g�����5�nW�����U��M/�g�_��x����G�����:F��x6��V���f�wX�M��D�d��+�F@8���9���������B�����[�]�����]���������p��g9]F��FA�+�=����L��%�j��! ��]Y����R��w^!�SX��umJ��W"��<��;p�Q����q@u��I�`��.5�/�7�BX@a|�U�
� ;����j_��|�x{T�HRh^�F�l�� �
�N���Q�Q��h�<5�x��Z,~�������h��t���n%w���g>���+=KS�w�u���	f��FZ��p�8��{����{f���P��6��������h-/'{�r�)��X�����4��	��X�[�/�'X�nw[F�Ok.�2�leN3�x�K��Q��lPmf|[��sO��2xv�so�����RO���Z��U�z�!�.���6mhow)�eW�
��GE9�P������s�,�t��z��������C(�!��c�Z���%�Zx~�W�]�kY/	��H�rzt�m�t�� �P���k^3����w�-���h�+D/J��%�v��N����7����u/
i~�����.��
��	�s��w���b(��x�_�Z�������T����EyC9	�'=?Q\��j�>(���/�o�.o�%4�Hl�����a�v����*jO���8/�����7���E���,ele��$A<�g�J���,|I�M'X��h��o�
��U����a��2�8�Q��g��t�2�/����%��e6�mw~G`��}Es���?��iy�~��j�1Kkj����9,=+��=E!�A�(������x��z����<m
�8g�v�|p�W]��9������d�����0��~{�z�:R�fx��A������l2.�����LD��H���	�`J��(��(J*1<EK	Sh����> ����N�1@QQ��PJ�0��iL�+i��)�t~f�1wt��f��&$,�H8a�>��O)"T!z���<��n�Snq���	(���e@C�sCO�t�222����a�1&�"�������$B���h�Tbx�;eC���P'��a*mN��IEG��7��I�;�(3���a�KhJ)�x����	�
<����b���p�>�`�&$,�H�)�2�%B�8�	(����n�Snq����P�T
z�������q�Da��3Fa�Cwp�>�`�2��[�0�"�gl�v��0�P�Tbx���Q��Q���w��I�wP�Tfx�e@�,9�3��H������L���6y��8��f��&bD$u��*1<L���B�!�y�l�����;�(J*3<ACP)�whi�\n�FFXs@QL3Fa�Cwp�h��)o����Q��6v��'8�OR�T*:��������f�56��1A�%PZT�����a�%`D��4y���<���m�3@��'����	@aH'��"T!z��������f�56��1A� ����;�4%O�t�7���R�������W��f1�e@~\�O5b��J[b[��E���8�?�L����*�0�P�Tbx���Q��Q���w��s���	(���*�e@�,9�3���=a�>�g����]�6��O1U�	@a@������Q�C(���[�M����%� ����;�44�.7J�##,9�	(������7�&���H����f�EF����"�0��<EK	P��w(J*?>-�����gp�h3��	�n����$������<�XsG����]�6���L��UdBGPR	�`J��(��(J*?>"����	�1A� �����;�
IEF����������W�d@���h�S��6	��w�,�6v��'8�	(���,%B��8���f�56��1������c*�	a�+O�D�2��}�:?3g���m�3@�&b�"@a@�(��OU��I?�^^��Yc��������$�X�5��v6�}1_E�����x��|���]��ml�b��s�c��W�tyvbQJ��V��(��s������J)J0��>���['��+���]��ml�b��I[#�]�����z+��km��������u?�]�V�uV����5+�18S+a��j�[���@X����5c��������a�q���-�.PC&�2Wa�����#��[y��Xd%NN
	Y,RHS�.����Re��'	���q����(��d�e��t������������&����g�&�gn��z���r��u��>U�~������j�����D2^�!�������B$�A���|���VK��JGF?h�geE.(�~��LJ)ph�������`�����R��9X���`�������<.?����&��~��W��ln�)���&��a���)(Ez88�3j]FQOXel��F�����R�#r�wm8�����e��w�/���Pa�q�)NP�E���0�)�@���ri<�w���H�r(��T�X��HJ��H"������)���R���*O/��:��q���e\��'�#4Y����/�&�7�p?�WU\��WX��+#)�����}+������M��wf�������F
$��3�1�=��
<K�� ���kR�(��������f�4l��[�0�>��g���T88a�x���Q��P�R+PT�BZZ(��(��(����h�:o��x�F���^���E����� ���Huc�'�q������]:�_iK������&&�',_�����_	�Sx���T�����km��m���y��'����[T�om��Oc��,��n >���R��m��
Z0}CS��t�NJ��,�v��5�~!�����)��~�;�f;��AVH�qY>�k�������z���7<�*�rp>��|Gq�|^�'����^{	�n#"�K���U��W��?�-�d�5��-�`S�s�1 zq��W��:&��*]7Z����a��Q�}����{�W�|1����i����E-��1�z&�v��|W����
v\�=R�+��GU]����En�e�-l~j�f���a�����F6��5�ju���Oa�[Gukp�%�U���"���4=ZF�J-=�n��E�����@W��
�D�)�iLL�xv{9���a��Mw�7�N�c��T���'�������%gRRI�u�\[Cwk%��I42�I#�r��`�Q\M���X�����n0��E������Dd��Z�M����'�����K_1��
�b�>�5s�����!�������)��]c��eUo7����^�^i���j��-����u��i&����]��<�����-���8���9���*������t��������O��N\�r�&��[���Q���TW���_�����p}w�����\ca}x��_�����]���{.��
Np3�^3^��?�M�_-���w`s���\����5�������m������c�i���4v����Y_�/�����Y��O���?�W�u�\Cso�x�P��p���Ut�_M�lM��s������e8a��k���"�A���|9�w1�F����el(���O��%��?�
v�'t���u����e���NH��4�[T���h�p3��"�����i��M�/	��K������)��97�g��]��uK��;CH�t��t�w���eB��s�94���~�mQ������5���l��5�+^
jn��
7��V�E���1�4Y�>,��������
/I����qjfk��r6�x�����sO���a�ff�����h��@���0��V��l��).��mm�yfp��Rk��I=���.�P�m��]���r����dq�]���g�[}�Q���
��S e$�����1��x[^����Y]\����f������4/G������b2�� d��k��������-�?��/����@�m���������OiZ���<1�hZb�'M��icy�%��h���4��x��~&���B�m�g�<�H��\��G�sZ����e�����cs�i���u?�ax�Os�F��������L�����q�Q��pEqv�����s��X�}5�COd���L�(,�Q��6���8�7�4}~��4[�{�tm���B�g��f��]���\h�_C�v��[�����	���\�W��I����d�i����������^w����<����?XI�x0;?�y��h���M��x��B�� �R�P����������������wP��mis��<�3��8<c^[�_�/�|W(?�7�n�����Q?2�Z��5�t?�7��i����(#��k�S���Q�q��=D�&��X^mQ��X��pJzdu��g8�W��N��^���j�6���:h�[;�����/�0��3��S��@?*B�i��HzPQ���j�?�_��v�����U5o�?��X6�>����@()�����/���3���JZ�m�Y�1�=X(�"������`r*Z(!kB����]����P!�=��.��jZ(&��`����PPqJ��3�hc,:�"���#�<"_0E��v����m�BJC�0H@3R�@-�
��a[��������c����1�Z��������FUz�������h���R�@�xL�a�2����?�����s����u%����A�8@2(�( �+rW`��h���@�������m�T4��

KEF��;hcf�(21G���|�*=����?�IED��#�����-m�YV���5-�`��y��v����EF�2��AA�R�@5�����FPJSoI��/�qQ�����#���b������p�a���h���
#
��0h�-����`9�c5-Z���h#!FP��3hc,:�"���#�<>o��G�9��g��[h�$1� �B����mm�YV�����>��S��a9��c5-���!�2��J
����h���R�@�xZO0�������b����J(%��3��p���@��!Aa[��5-�`��y�!vf���eUh#!FT�PMm�g�6#�������g���v������"[h�,1�<sB�������jZ(/���e<����3A���CeW��-Z��w�m\�8�����h�/���?�IES����<i?��_�<���kn�k�`�����Q8��	P�Frs��4��Q�AV�>�����fs�`�}jZ(&��eUh#*�tP��;hcb(8�h�����yQ��wm��[h�,1���5%�������h�,2�����	�h���#*�`���`}������8-�������*3��-�y�(��;���u%����k�`���Z����F������"�,^�&=�������`eUh#*�t-[@��0	@qK�xL�g���v������"[h�,1�=HA�B����`�`sR�@}���'%vA���CeT�.��R�@5�<�Q���J��������Tf���#��|�a���F:E��3��p���R�@X���
e����G�9��c>�-���U�����P`P��9����8�h���	��1G�9��g��m�W.���,d����1�IPxWUhbH��$��5��Y7g{dw�^�����_����A5��YJN����3&���;��qm�����,�8v����E{<��y|�p����zI�9]��o�x���U�.���c��=(2�H%�����6�9W`�c��7n���3��Hv�9�M��X�f8H��;����f����<RRQ����j�3��sO!A)��8<qU����K�J�NT�'�T��������:�[��lr9J���� ��J]Fo9�@�����`���>�
�c�\��L�w(�R�U.r�9���[�������s�@9�����viw%73�|�<�<cf��})^��@��%`�����P�G*���3]���������s��Q���7��D���v��zf���X�iw&[��vu��3}�r~��xfh�������x�?�p��|>f���o��������'V�+5���#�
'����'���Q_��K������FO�%���.����'���9��.����&��\��b�����IE>yw�+�"���A>?�����R]g�3�����zm��x\���Myr����C)Y�mI�IV�t���]�'FO� �a�&�G�lq�TtW36�$k��0�4�E.q�R���T���S�����QQG3�Ss9�9�R�0���h3����Fo9#�QQG3�d�sp������X�#'�����M"��@r������O>c�f���6o8����gp�#A�85(�G3�H�7�&�u�<��4���%�v��]?�`��y1��v���+��e�$�F���Cj�����
>��a��#k"�AU�

���cm�(KEjQ�����/�qQ���o
�dXcs�
3���PKkgd����"�kB���U0jZ(U*�R�EQEQEQEQEQEQEQEQE��w�����b�_�8W���i��s���q��@������t�Y]�#R��8UQ�N��7W�����:�S8�E�����������n���Q�����6����z�����/����8�<��U��m���j?�5_�����Y���z����>���bj��
���-]���n-4ic��H\�1"�$m^y������4��}�d��<:�.d��_��WZ��-v���[���J�(Y��p��UE{'QGD���4��FGKX���
�� ��-�z���WT�n�������X�2���� �	�_C^�Ey}��vio}����UP�G�	���}����2��#�g����
��h���C&�k�mfm�8��^:W�Q@Y���_�5x�/.��fM[��6���@�+�Np
O���|>����|;���W7��i�Z|b@D�&92F�s�ze��.��-O�������Yy�v�cO���L�
t�,��t?
]_iZ\���(����18��S[4P�x7V#����U��5����zF������w� �%��sn����o*1y����"�)���=+~���>
_����\h�Z>��[�"�BMs$�����5��|@��h�[�WJ��r�i�������
g%Ns�^�Ep~��K�w�~)��$��V�me!bE��������~���	��6��Z.�l�h�1bX;	�U'��T��<�\������M&�il���I8�X�C�3�&��ww���w���k�3h�[,�}�L7�H�����P��E����������n�i,FMB1���9�:�^�ER����"c`�LUM[�A��_�V�*��� �������- �K@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@�|�.?8�jco�x�5Z��c��M����<�@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@^(��OU��I?�^^�����_����A5��yG���x���@��+�<���(��(��(����S�����uo�Uj���!k��������c��QE2B�(��(��(��o������7'�<�]����]���nA����o�?�Q[���#��TvTQE~z�EPEPEPEP7���q�\_�A5���G��F��<.?����&��~��W������QEw�Q@Q@Q@(�IJ:�����������m]Ur����������m]U}��G��>QElPQEQEQEQEQEQEQEQEQEQEQEW�i��s���q��^�^i���j��-����w���x�����������y�A��d����44R�c3#�a��$GCJO�6'�5h���<G�A�S�$�?�,���O���k������������W���x������I�4�Y�?����'�����a��c�z+���K<G�A�S�$���������������V��<��`6'=�5���F���p���c����+Xk�R(�e�F
�=��RW�hg�7������Ez�`�����V�*;{�.��-f�d�Wtndu���<m��A7��H��GR�!@#�e|0�lt�7����[�]�|���)������Q^}w��i:kk������#_6{x�����������=cQ�$����V6W��|�����j@*pf�{t�
�+�����I�I/�|Qami����V���d��0�[�H$p}j��&#���Fh�W6\��k�������H����O�Yj�:��Q��Ep�s.6��@*}�����mMq"E��U����%�mF_]����.�#4�����8�"�>&Z�{��\�����[R�hY��8u�
���i�(u[)$v
��(K��W&�+x����"�g`�~&��C��^�{�[�>}��io}{��<�bE�����W}�	h�*[e����[9x������Fzw�
Kk�{�D���<D�'����J���o/-�����UB~�5��E���h�V��0<�[�_�'q}{�jz=����&��y��n�@�(��
��s�*K�K���70����@�L�^����7������G���SJ��pE'�����~	����?-��,x��U�����P��i
�v��B��M"�o��'�Eu�i�3yW�����
�Y�=pMx�7�����/�wZ��6D�>�Er>�\�u��[�
Z|s����`l��
���D��y��`����[]��C��O���p��5yG���w�KQ���^[��C�C%�����B����^�
C������(N*��� ��������=q�S����U�tpt��)h��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���\q����g=j�Av��'�	>b�����J��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v���w���A�<w����~�~��Pu��������V��H���Q_��QEQEQEQE
��x\���Myr����Q��������	�._�>����f���Q]��EPEPEPJ:�R�����/�&�7�p?�WU\��/�&�7�p?�WU_aC�Q�G��Q[QEQEQEQEQEQEQEQEQEQEQE��w�����b�_�8W���i��s���q��@������uW���%kx��	��Y�Z��x������.h�'�>C����|�6?���5���e���s}_��c�q���F�����&���������C��g�{���|��o� ��n�����8�����6�T�����R�w�W�`��� O��{��(�H����7��G�zE���^M�x~����6�7]6�s$Q����v���j��<J����o�9=��������W��G^}���������p^E���uhcyi?����y�\�����P�xN
/�xSH{
.M.}���\�
�3�i�c�.����.m�mFk�)���[��;�����B+�(�9�I<�rx����KI&�d�� �G���}�����msZ��4�5�����"##N3�s���(�&���t��I��$�����D�����*�b���w������[��^#���-�s�����%�+��h0x|���ss*�ba��[9�:sP���n���|%�6����jw��=|������~����|5�xwW����s�%��y;"��c�x`;W�x[���$����k����di�n%S�;�Xs��E{��������zM�����n�,~�'���/i>��{y�]Ky4(bI6`[�c�����Q@Y�_����&���N	[O���=F�@�G9o�������z�PA�E!�@d(�\sT�o�?��[A��SV��s�W������t���(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�
�i+�IH�����V*��bH��1*�|�f�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v��6��>|��;s��\Uv���w���r���_7��(��������*;*(��=@��(��(��(��(������/��������^�{�����^\�t}+�����u
(�����(��(��(��u��h�?�_�M�o���6���_�_�M�o���6���������F

(��((��(��(��(��(��(��d��o�D����i�]At����ULn���x#�&�������*�5�G�rK@?�y���x{��<&�eHo��z�����'��s����������E��P���d�s��d:����8/-�v��*�k��3��~>0���/>�n~�6�3�~�+��&����WMG��i/�ZU�����
�������zI8�P�wos����.����c��3��ww��������ng��Hk��c%��*�����/���i�/���s�\�.��W)w0VW��G?_j���+�y�h��9>���G�
��z���/���KY�Q{�����}��?�7�~h��\�%�42�z���n#���Td��d����
�����'��^��������?��S�����{�{x$C�^h��EQ�'��C�������Ze���UK������'6��=~��N������[��G
����W��}n`�������|��m��+���j��-����z��\�4i��#>�������w}���EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�G�����*�EQ�|���?�A�%
sc>;�W��c�l�0s�����vS��8�)����bj�)`Y���Yl60yA�yk�7\b��t��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�������A������k��?41jci��*�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEex��E=W��$��MxE{��?�S���O���W������f_}�J+�<�h����J(h�����(����-u�][��Z�����]�V�uW5���{�f��QFh�Q�3@f��]����]���nC�=�����m��w������y�����V��H����Q_��QEQEQEQE
��x\���Myr����Q��������	�-R6�{W������-dz���6
(���#��
(���#��
(���#��
Q��43��>��e�$�F���Cj����e�$�F���Cj���(
>��a����+b��(��(��(��(��(��(�
���e�i�����wV��XdWpk�|u���5=\�,K��^jQ��XB�������<�0G�w�(�#��������!��m��A$R<����(�t�z
oV�uOjV���6�s�O���P_q�X^]Esweo<��X��%f@q�	g��'��ac�x��Z��V��������V���0`:�q�|Wk������X�U�]��{��@_M����F� �r�������F����}}�k��jV+�R��h��#�����e�v��s���\��=��(�wU���K�k�1�\j[[�����
v^)������r,���mx�tea���4���m��i�j�����`sf�������('��9�:��
5;��m/RKy�=� ���Q��P�N�G��3e��cR~���k������wN�l�6w�BpA����5��|�c���W���n����X�7~d1��n��q�t��o��	���$�VR��}���	�z��5k[�z��$�m��++��nB�$c�Z���g�iWn�n�6�(c�'2����~	h�������CG�p�i7W����F2@�&�4�
�K�|��������\����a\���}#���V�g���"��2�	^<7���=	�`DX�R5
�0��J�m;�NcV��n/�(��|��f���,M�o/���#��~f=��ae�y�?y�s���Px�-�,R<[�<���.�l�<k��n<���4Po�[y>W��n�����C���F��c(�x��Vh�
�X[J�^<�G�x��6P�<��3��=jz(�X��)�#����=�D���WT�����#��4Pa��Z!��7��O�0�F?�I n<���(����*+�����J�6�0����`�=��E@,�[�<'�3��4��[D�c��S�A�j�Yt�d��X����q��`��|�/����_���@����H�<�gh�x�id�����y*1���Q@(��)���v���,m�c*��9��{��V(�
���F�<o��?:�l��B?��$n?�Vh�
�O�0�F?�N@�z�t>�m"��y6��x�Y��+����	^<�����zQe���~�9������+�ao3$x,?1���lm���V/����8�j�6BDlG\�iim.����}��Z���g�#h����q�?�>����gY��=*j�%���K���~c��)���<����s���P	en�U0�9;����}�j�#�u��1�~uf��4�e����A#q��t>��"1��r����4Pg���TW�!�<���m��H���;�o��Q@(�����w���[����y��Pd�����c��a������[y&//����_���@����H�<�gh�x��K%���K���*������(�������=���&3*a�rw�{��(�i���	���G�@���&�G�1���~5f��t�c���)�������i��A����4Pw���E���.0w�;���q����s���OEWKx��#�`A;���4�d��c���
�����@������������CX[<IG�L���j�^Kivo�;*�������
p&d������OE@�P$�eLHs����i��[D$x�[�<����@�O�X�1���8�h:}��E���������Y��gDV�*�
7?ZW���Ew�,�w��X�� �h��~�9����;x��#�0 ���^�b���}�q�$xWa�����a	���n#q����(�i��F��;F��$���f���v��x�Y�� k(�L�����B�@���1!$����jz(�v�o�7���G�B���F����q��Y��+c���nq����}���<yTQ�����(��6�H�������z_�A��?g�3��=jz(�X��#:G�`A;���4�h��c���
�����@��l!1y!m�n=:O�h�3�����j�ZKiB�;j��������L��;�o����3����t���2��<z���[G�dx��[�<�����#���&T+�	��Pe���'�c�>7
��~4g�y"//�
��_���@�O�x�<�(�x�i^��GVx��q����(�A��?g�3��=i��9D���=���Pd���GT�
�7G�@���&!�N�7��Y��+6�l�,f?�I n<g����i�;j������(��y���q���N�-�pgT������OEW���-�#��U�c���O�H�5���7q���(��-���������C���F���P�q���4Py,-�eg�%@�=N���q����s���OEWKx�2$xf�N����M>�5uH�a�c����Xi����c�����A������)$
���5f���}�����E��1�~t���y���q����S�@-�q����s���;h�l���|���EV]>�"x�<#�p�y��G�}���y&���z�uf���}�����S!F���K%�������|��?�E@l�7yO�g9�z�%��r�<;g'q���X��1<Ai
���W�6��$�q=����.����s�G��z��zI�������1t�u��s58��	�X7�F	��g�I*��rx��C�������EB��S�0(3Hc�[��1�L��H�iw���l��p)L��[��p*:(��`��rO>]��|��p(�3�n�j:(��`��rE�D+c''�I�H#��|��0)��rG�sK�����/}qK��S��q��j�&�w+F�K�n_�x?��������:�[�p��#���h�Wm:�e�����^�(����	x�+���U����d�}�q�wq<��26�;��Sc�.�����6�F���j�E���>��r��g2~���t��4�jWs*	%�F�"���U��Hvyw->�w$�����$���g���J������7l^�LUZ(�p���O�i5+���T�&7��?
��$��<���v�u9���_���/�?6Pc=��_;��G)�����GNru���'�FI���
�����K����2i(����g��3�.NsF�)(����g��3�.N1FN1���=��k��b����QG�����g����l�e8"' ���^k��;J���q�9��^�{�����^\�t}+����OVoI�K1��������|�������o�,��m�:}q������s[����I���cGn��^��VB���n_�p*�E��e�r�,�\���P/�D�(��e�N�����S��p�,���n����;�����/�#���a�F���j�E��d����}����cn����������$�
Ah�����(�K�]���?��)s��L�q�����8����O��_�[y&//�-��_����_�M�o���6���������F
+6�l�$mQ3�n<g�����]�����~c�����D
e\	�3  �q����V�9�SI;�~��z(�z}�a�G����c���]>�bh�#H�y��Vh����*��uPEPEPEPEPEPEPEPEP^i���j��-����z]y���'1������tQEQEQEQEQEQEQEQEQEQEQEQEQEQE����z���M�f6�A��<�������7��q�MP��Ge
9��s��M@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@]�����17c��������4����*��@5b�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v��m���e>�y���Wk�����\|�C�v��|������JGN������� ��(��(��(��(�o��������k�����z���x\���Myr����G�7��(���6
(��
(��
(��
Q���u����7����������~�7����������
���=|((������(��(��(��(��(��(��(��(��(��(��(�4����5o������.��N������[��G
��(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
l�1�'�u6Lym��q@Y*��
�Dj���=Ae�X��)����q���
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��+^��q�p�J�:���5Z�����K��6�����(�4f�
(���3Fh����(�4f�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v���W�����w�<W]����]�f���y�����V��H����Q_��QEQEQEQE
��x\���Myr����Q��������	�._�>����f���Q]��EPEPEPJ:�R�����/�&�7�p?�WU\��/�&�7�p?�WU_aC�Q�G��Q[QEQEQEQY���4�
io�����Z��+c$�u'�s@4W�?��gU�x/���n~���-cq���#�������?�>��[�5��rTz����4V�|m��������g�������y�����(��(��(��(��(�4����5o������.��N������[��G
��(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
l��FFE:�!�lq�����5�5��5*��*z����j����Q���O@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@'���	����
�a�'p����k���+������7\������*�,��Zu��&���s���Q��5��(���!'����G�%��$��W�+&��>���������'�����k_����_����k���+��E[����}�=���[��k���+��Q��BO����h��x���/����vk�Q��BO�����J5��I�|��VM}o�?%������o�J5��I�|��Q�	F��A	?���
�����x���/����vm��M+M_�j6���P|����a���*_�^:��?��?�.�s]������������"����;#���x���������?�x�����������+O�G�~�g�����?������t���?������u��Q����������t�j���]��t�j���]xeh�;���{��/�������G�/�������^E�8���{i����@��v��|G���O�Xx�;����`���h
3�1�}+�Z�?������X���&6�Z����YJVg��E��AEPEPEPEPW�?�S���O���W����E=W��$��MxE}Q�9z�&e���(����((��(��(��))i(����-u�][��Z����Z����������{�QL���(��(��(����o.���7 ����\Mv���w����{w��o�?�Q[���#��TvTQE~z�EPEPEPEP7���q�\_�A5���G��F��<.?����&��~��W������QEw�Q@Q@Q@(�IJ:�����������m]Ur����������m]U}��G��>QElPQEQEQEUK�.�S0h��u�i|�D�#����c��[��2��#�[�'��+>���kx�=�A-�d��W�?</��M�^}�W�*.���2Ej[��:��X~ �����+�\���K�������.�bc��:ds�T:�i_�U������������>�X���6�{����ir^f����]�����t���������k������_[����N�>���+��4g=+��g���\��+�wz�h|������w�����Q��?�/�v)�Ou�-+Z�R���h��F�3��^�=����y���r��E�G�����TG���(��� �����\�E���>x�E�T�%��<=���;�~�����u��oo��z��p<W�i�/��+$p�Z���;�p��~���y�u�x����B�����W�"��� ����\��q�FGL�N��TdW
�;�W����^X��5����.2���B;�?R+��%����c�I��},�R���DRFX1���x�"�=����<y�������
*1=�������_�O�s�j�u���|//�c����������6xS��(��@�`gzv�a�4����5o��������x��v�j6�}n�';I��r?
�4��9�[�����p�K��(��(��(��(��(��(��(��(��(��(��(��(��(��(��Dm��)��������Gh�5�M0"B��#��*j��d[8Vc�����jj(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���������	�$Ibr��[�������O<5�AD��o�������2j��/��j�}2�2�w����MY_3���<�������G�'��������k�(�������O��^������	����(������?�<�������^)E�4;�����J�e�|�k����_�O����M��xk����~�����������A��W���g���y���
'���&��O<5�AD��o����QG�M��� ����_3���<�������ZV��k~o�]��y8��a�9�P=
x%Aq�
o������s��$}�z}�q��D��8�������I������N��������.��	�G�-��]?�/�&�������F���;?����h����o���������4�������"��h���w_�������3��+�o�[�6����$_�M���m�At��H���?�����A��C��?��f��������.��	�G�-��]?�/�&��lGu���kP���>���f����o���E�������O�������~?���;?����ia��������U�i�O�j�i����Z����5�q���=Et����g��B��o��2���"��QE�FAEPEPEPA�E�y���#��v5�Z�������}{���u��QEeQ@Q@Q@{��������?�W�W����Mo������w�����F�~4z�Q_NwQ@Q@Q@Q@^(��OU��I?�^^�����_����A5��yG���x���@��+�<���(��(��(����S�����uo�Uj���!k��������c��QE2B�(��(��(��_���o��Ze3���Z����|Sm���2��n�|������JGN������� ��(��(��(��(�o��������k�����z���x\���Myr����G�7��(���6
(��
(��
(��
Q���u����7����������>�7����������
���=|((������(��(��(��(��w�+�/���k�������cYq[�Nv>��Yg�a�X2�����T����~|Q�����+�Z�mn��df�b1)2`� ��z��p|q�?�&x��_��z�=���g6�f�7�D�ql�>9P��=j�Q�uE��KMJ;k�U��e]G�X�������C[�_W�n�"����t�.�d�s�Oc^9�K�����u]F��6����:�����q$G;}r���E�>���5h���ip�yA$������'�����{���|S�i��u�	�����f�,o(������?��
�c������'���I��9�4����m2�i��Q�������~��%���hZ��z��!{K��z2����u?��Y6��>���3X����������dN�{H��$z�K�C�Dx
i�����Rzo*�U���F�ua'YA��j��V���<e�����i���p.���]�T�����rO������>"k:\�V�:�-._�H�t�[����3~ #����<�|G
��R{9�0?Pk���,P�$����ba�� �����[x��GL�w��)V���1���~��c��}���m'�>!��D�E{��`u��i�[��3@��a������.��������5�^-�(<�I95���>�
E��i�,������o,abx�0���_�����_u�<5���K:!��������>S��''����F)!�/���!���������gR��=[���a����9����O�]|�m���=x�Y���m+M����Emk��U���w�����b�_�8P[�h���n���&����$��(�����	�^k\C0�.nX�O��.?>�=^8'R����EO�"�p"uk�flmm����f���\y;>���g~��=1�P�\4h�v����~o���@��ve)v���r}zR�f7����/b����=] �e,�L�s�(�R$
�������6�^�f�� ����.NC�\���P`�0���	%�/>�*�Y��e@�n�F�S����x'i�-�*�e/?�X�� �.7��1�>^���=i#�uf�v� �0}zU�(��p#uk�flmm����yN�������s�Lcf���\�V���;�b���A;2��t@E9>�b���1��.XG��{L��H'Y�5�2�
/�X��+$
�����%m>�(\YM��'!�/�Y��+.*������lb������bN�����@�	�`�t��@�sJ!�\o7,c�|����z��Px��Y������{�~�8��^�����(�V��`��894N��e�b�o	`$US�:oz���1"���.w6����A�oA�lc#jz�$�]�n�PN��Ja��o,9��.?>�=C:�Y����!E��i�����w,�RQF�����@�Sv��>�����Qv���}��=1��EVx.P-��Q�!���J�N��[�U��S��E@!������/b������:��wE��Pd��cuk�f`6�E�J<��'o��~������1��EVh.H�v��9m��R��We���q��P��qr�<���\~}i�����9�\��X��+$
}��FQF���������bFb��U�(����
.�89/�rG�1C�p��n�
�1�q��Vh�
��"��2(�T"���3}��[��|����z��Pt�uv/t� �1H�\,n���|�Q~_���@���$���~������1��*����s�_���4Pi �m�.�0�"������n��T���������� Xf�k�d�����Zlp\(}�n��
J(�}zU�(��p!e7nX����/�A������%�.H��*����*x�R ���Q�H��y����RJ��q��k��2j��/��j�}�8�����''���O/o�7c�d���@*��1��E_*#��4�YH�@F��bo������j:(�_�a����
�z
��Et�*�<|��er/���c��� [��4C@7��i�Q����gC�B����y���f�l��~P�zw���|U�.��?�U+��dT��f@���AscB����}s�����#=�n�AU20�=:�Uh���������1�[�l�d�1$�20��������/��4�1�7{g>��*�r/��s��d��h�R��a#�g�+��4��X���,�F;���Uh��M�;��-�o�1����'�9�sl&wk�
��6��j�r/�����"�w6�_}�rnl�20�=84���1��4�8��?�^�9��������w0�Ai�(���Uv'��q��#�+�|Xx��b�!����y�M��x��c�����_s��I�b�g��IE|/��������#���JJ(�����]��:~4g���9���]���FFzRQG������.���J20x�����K��_�g�ksB���5�;������:U�nZ�|��l�]*���#��v5�^�9>E�u��bI�`�,�0A8v;��
<Te�EU�PH�o���Zs?���>8���f�y~cc>��BO���h��mS#
�^�9������
��I	$�.��R	��7�mP��y��9��J��\�.N�B�[DF��c��]+�:�D
r�$c�zUz)�?��x<���A��l����?A�����}�y���7������������]a��Z�po��kI���'���]�\1�q��Nxgi�-�*q��?�Z���c��a�\o7,c�������$pN�����HEO�X��+,'V�vf����/�G�q���[����8��1Vh�
���n�Y~�_����	�����)��X�� 0�n7���>^���=i	�R�t��8B�1V(�OE,^�L�
8�$�+(�}+����%@�+��^�����_����A5��9LS�/S��[S� �=��*F8]��D�l�>c�GE{<��l����������������o�d�(��3��B����y�2!`D*�7j:(�_�a��<���%q����$�1& ��r/���c���U�<��7�������i��r����g��_\"�����@�I���5^K�fh�X��[,�wNOj��������n�����n��h�����a ��6��9�\�	���eG�6��9��S�_�a����Q��+�^�7�Pda�zpy�K�u���������N3�*�r/��s��"��k��Dcn��}q�S���u@�IV��������Q������H��6�,l�1��w ���<Q��o�o�{6�����sUh��M�;��-%�����1��6�����y���2�G~���[p'���+�����o.�m�7!����_;���)��e��8i?h�}�<t�h�)(��y���#��#�������i//��s1r3���JJ(�����]��:Q�:~4�Q�����\�J2=)(��K��AvEv@����#=~S^i�fk$e`6�����zU��x\���Myr����G7g�H��z�#����$�2�6�N)�4"�a�F��w�~��P�]�����4-E�E~3 v$�)^x�e� S�F;��W��3�������ca����<b��
����=j�s?��	<
�Z�1������H�B��kDv$���������g� �1���b�������\t�<���K4B%�����W�is0����y>i��x~DP��n���k��.<��k}���b���+��e�$�F���Cj�����
>��a���ApbE[�V�l_����	�n��L"����QZ�@��n��	�������	�r�t��8���~=jz(�p\(}�n�\����������bFb��U�(��,j����:�(����/���	YA{��$�N 3���D�Ar>�����@m�������>����i�[J�����RF>�/q��Et��W���3U����Mt��c��5a��Q"�\,q���6
���--�������m�>�P���Y�]���N��1�!�s�S��V������9��O�'��=>5������@Eq���p;WI@Q@Q@Q@Q@Q@y���'1������u��w�����b�_�8P��EQEQEQEQEQEQEQEQEQEQEQEQEQESd����`��l��##�Z��a����]����z����`d@�cR�c�O@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@����������^���S����3T+�i���)�l(������(��(��(�w�_������W;���u���J��EO�������
(��
(��
(��
(��6����]�v}6���^e��f�E��+���uO���"������Y�a��
(���
(��
(��
(��
J(=(�5��/��������`���������/C�l��(�((��(��(��+������]a��Z�r���G�zk�����/�x���5��������s�(��(��(��(����G��z��zI��������������	����?�/S���8�Q^��Q@Q@Q@%-%p��������:�V�?�]�V�uV�[�p��)�QEQEQE�|>����6e8��k�������~����6��<���G��+|���t�����(��P(��(��(��(�����<.?����&��~��W�������q���/�J�p{3z]B�(��`��(��(��(�i)GZ�O����}���
���W����}���
������(�#�����(��
(��
(��
(��
d��q�<k$n6�:��==i�P���t�6v�Z<�r���������Ak�����jo�k!NBj7�"�C�IwX���UI8�������?����&��Yi1��v���F0���~�SQ�DV�N��(��(��(��(��(���N������[��G
���M;�NcV��n/�(����(��(��(��(��(��(��(��(��(��(��(��(��(��)�F�u��S_>[m������K(]�Vd�1�=*j����^~|��~z���QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE�Z������}�����W���5?����C5B����������(�
(��
(��
(��
�|U�.��?�]s�*��_�����T�Nz�(��P��(��(��(��(o�
����F2v�������Zy��-�������_�q���?�~��0����Q_t�Q@Q@Q@����������}hk��0_�cY�������AEU�QEQEQE�?��[�����-^9^��#�=5����������G������tQE}9�QEQEQEQEex��E=W��$��MxE{��?�S���O���W������f_}�(�h���(��(��(�����8}O�B�_����U�Z�������:�[��G�QE�
(��
(��
(��
��(���'vPc=��\Mv�����<���o?��o�?�O[���#�
�TvtQE~z�EPEPEPEP7���q�\_�A5���G��F��<.?����&��~��W������QEw�Q@Q@Q@(�IJ:�����������m]Ur����������m]U}��G��>QElPQEQEQEQE��$���3K�����=�K����	�Kr��]�:���Z����?�7�K��S�i��R�O���m�c{��Ug�G�0�>A����F��)����-�sm��\1�?#@���6��k�����-"X�B����2y5v���W:n�ig}&�qJ�]��i�ub=M^��(��(��(��(��(���N������[��G
���M;�NcV��n/�(����(��(��(��(��(��(��(��(��(��(��(��(��(��)�c`:�������h;5d��\�e@	98���%T��Q���@a������(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(����d����_��
������������4��|��6QEh@QEQEQEW;���u���J����W������Tw"��s�QElr�Q@Q@Q@Q@~e_Z���8��^�^e��+��]�[!��a��^�+�.9�~������0��+�N���(��(��(������<�]��������
w�F���k>���:��(�����(��(��(����G�zk�����+������]a��Z�������#Z?=n�(��;��(��(��(��(����������	��w�G��z��zI������<��r�<L����QE�PQEQEQERR�P���Z�������kS�����uo�Uku���
(��!EPEPEP]���_.�q�eq����Wm����~�N��wv�<W��G��+|���t�����(��P(��(��(��(�����<.?����&��~��W�������q���/�J�p{3z]B�(��`��(��(��(�i)GZ�O����}���
���W����}���
������(�#�����(��
(��
(��
(��
(��<��l�>,K�-OP�����>;��m�����6�?���A�_����o��7�����0\Gt������� ���x��:��k��2hz��F�T���;$^�?��9����|O�w��kn�;���}.��y����0�@��u�|E��#W�M����3��2����PYY����gg���8�tU~U=W!��[����x7@�����%�5��F������X�>>�6��
7@����[�Y�v:��#4M �V�����I�����<G�x��:�`�$������U���U�(���W4)>%6�n<G�F�����������������@��������.���m��������V��H}O`;��O��xF�O����G�_����K���H�tH���4W5���x#���@�R����g
<��T�>����x��V�����=�
�U�����{ ��7
���z���i��}������� ��g��cb��"�K�|F��%�>��}+�*�=�����!8��~R@9#�`Y�4����5o������K����*�Q��Kmu���V��w�����b�_�8P��EQEQEQEQEQEQEQEQEQEQEQEQEQESd������S��c�l�����`���,S�]��$b��,�5�%(�5���=QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEx��#&��_r��f�U�{�FMO������P����/C������+B�(��(��(�������WE\�������*��>���+c�(��(��(��(����~��J-vxm���k�Ey���Z�`8br:�����_�q���?����0����Q_t�Q@Q@Q@����������}hk��0_�cY�������AEU�QEQEQE�?��[�����-^9^��#�=5����������G������tQE}9�QEQEQEQEex��E=W��$��MxE{��?�S���O���W������f_}�(�h���(��(��(�����8}O�B�_����U�Z�������:�[��G�QE�
(��
(��
(��
��n���2���5��m��7�~w
�A����o�?�Q[���#��TvTQE~z�EPEPEPEP7���q�\_�A5���G��F��<.?����&��~��W������QEw�Q@Q@Q@(�IJ:�����������m]Ur����������m]U}��G��>QElPQEQEQEV��)���lf��
9b�L%������m'���=@������4+����mO�?fD�����q@.�����:��_k�����/=���GhA
�w=�j����:��[��x�!��1�	��d����>���_u��4�o���I��H-�;Dm��9�_Y|e�4��:��-i
�M�!���	��;�}���+�"�}F����]�d�1�1 �q[5���?
�_M������r1�h�kV��*�b$��������=6}(g�>�2��>[��0�{�����T�/<Y}=���=U�F�P������t�*I��O��s�.�����M�e�
6� Vh�dJX�s���z��
_�
���#�u]7]:4�8�����\��G�:��\������~&�����#���r-�[�S���W�?�����^(�����M������f��eG&���=F6���a���^%��|=7�<=�]�E��g�u��GA�'�P���������]�������V���������c��Y}�J��|'���F���95����#m[�?~<��?�s~!�����~���~�M�M��_J��!��{'���3���W�
#���k���7w��I����QM���i8>�!��r?P*���]����@h���{��4��
]?�����
�|G�x��.���x:�A�aT�u���`��^9l���h�SKs��	vII��V|�q��������YL��k����#��?�F��Y�XH�H��?\`�|�+���<I�y|,����n��-��4��TR0�)���{����d�K�W��6K�PO����+������h�E�-cc{t|?�/nZ	��U�s�+�<-���i�%�n���9�GV�NO�\V��'1�������>(����4�6�o7
��o()��Js�+\5��0��"������b���+��d�� `��!9=�R+�yN^(C��	�N*�[}���*3w��1����4)%�;���>�sVh�
�=�e����
�����K]}�8�����v>����
��fR(�|��	>�b�����B�		�����@��y,LP���G�pG���T���2w0��*�Y��*yqBI6d#��W{�0	F>2L�~1V(�]}�8�����v>�����7�*0v�������Y^�����8��$$\�F��'>T>f���c���EVg��H�/��d8N)d{����	�HF~�b�����0��a��.wc��D{�1B>pD��n1V(�
���_��q��Br}��=�������y�}qVh�
��<�"(|��<��>����*yqBN>`d#���EWw���Dc�$�A�������^N~���LT�Px���y�B�����4����2C�>P$$\�V��H�����S������ �Y�!��q��*V{�)
E	s���p=0qO�g{8Za�
��5V��_.(H�7n�����Z��X�1g�;���OEWF�3�D#�9'��R#�2(A��Br}��4P`��K�3#h���<�DP��9a�>��4Pg{����	l|��F���bP(�q�d ����E@��1����o;���$ovY����������V(�
���7/!���~�qF��'>T>f���c���EVg����P��p2LR��`��$`n�!>�U�(k����a�������)���<Q��		>�b�Q@��2(A���	������K�&F�$8?�*�X����3<�0�\P�x<��-�������@����GE8�2}�R���Fqy9����}1S�@�����Apv�!$����o�(C��! �x�4Pm��I>T>f��q��(g����P�9���f��#�
�\P��n�!>�S������#,\�LT�P
�Fr8�\����q�loxC��B_�l����Vh�
���S�&F�$8>�8�����3w#�8��f��Md!���3+}�\��sTTE��w�W5��5?����C5B���_"��>Ro�z�^^w6�t���C�������(��}���!�]��$�������v���E��s.��C���oc�d�(�����0��L��W�9�a�E������E��3��+�2�9_p��������Tp+��lf�,�?�q	;f�+��W������Tb������	a����[xo(n��3����� �{����b���V����9��b�idcc,���� ����e��wy�y���X����W��W�\��fD���i��7�~sK"X	#�9B~r�@����E��s.���hO?�������c=(T���<���D2O|��Uh���2�Z�,I:y��6�G���MD�6��4�l(���5^�9_qs.�G�(��hc�8%L`)�NFs���0O�^a�����1��=��W�W��n�q��������?��/�h�=O�IE|/:�U����!x�4q�i(��v_�������q�IE��������=M%s��_���������9&����u����0�������f���	|����_nsTm~�
�)��1����j���#��v5�^�9{�N�Z��"Y��T�1���00=����J�&��s���G�j��2�;�6Y��y�y[xo,g?L�����I��D`�����G2�,F����el��c��4-���e�����6�����\��\���DsL_���=y�+��t����?91�@����E>e�.X)g�(M�`����~��Y�*Y-���*���1�~�zf�z���G�zk����������#Z_=I��*yqBI_�2���+����#b�~1V(��;������!~�s�LSc{�������d''�x�4Pe{�)��q��Hp}rqF��'>T>f���c���EVg��H�.s���,�v|��#qi����Q@��Fqy9����}1H�ve"H���$$�q��Ea�����������s�?t���dC�a�n9;Fk��Q�"���^��&�"��)W�-z�.d�8���}����D9;���Q����g��<�e�z������PD^^C>�t��2�9_p����mw'<�h"��{���er�����;����]�9��s�^�(����W�9�a�!���x������m������IG+��8�U,���I�2��r��Q�<�������vR�9h�����5?�]�V�uV�Qv��rW��R��P��b���!��f��>{�q���3Uh����2�Z�,���p��+$�~x�D�0�$�	y��`���j�r���]�,����c>>��m��?��"X�O*y���D|sUh���e���b&�G<�3�������?h��?���yCv�Lg�U��W�|��i�����F1��@����+��
F#�d�B7��#��k�����M����(1���Z��&V�k=���������N�i�:�G��%�o2��J����8�SIE��W��a!x�4q�i(��*��/�/�q���QG2��./��=M%s����0���_�O���S����5����v4�0�#>���*��<.?����&��~��W���������Y��l�)�v���~i���g�K(�t ���j+��v5'u����e�*c{��WK0���1�����j�s.�r�K?9B�1�s�A���Y���yX����~���S�]���K2��M8P~B#������"d�`�8 �s���9�`�1[_�dK/����m��4�[@��M36F�����������`>��rd_��`�Ux�?�.v�������K���'������������_�M�o���6���������F
+3�yHV(K���)d{�������B0}��V�3]}��#FX������C��'���*z(�oxC��B��l����B������I��	���E62�5��B��i�Q@Q@m�_��_��o�Mik5��u���2yk���z8�MqI��[�����m�Kkqc5����S��.w0���
zX��������`{OH[;���Uuc��J�}k����'��xM��om<�-mu��
��']����W�����WL��P�R�[dw��6,����>��Y>���'���_;�6��d��U�kZ��R`z
Z(�z��t��(����
0=(��
0=(��
�M;�NcV��n/�+���4��9�[�����p�K��(��(��(��(��(��(��(��(��(��(��(��(��(��(��Lm��)��ca�dMGh$[8��2�s�9��$��aM��cQ�NA����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(����d����_��
������������4��|��6QEh@QEQEQEW;���u���J����W������Tw"��s�QElr�Q@Q@Q@Q@^(<Qi�c��r?�i�zp�2�{��E��N�����<�����������g����(����(��(��(��(=(���8��`������C]��������v�����
(�����(��(��(�c�����u��Aj���?������X������??�����[��+�����(��(��(��(+��)����'��k�+��Q�"���^��&�"��(��O2���QE{G�QEQEQE�������������Z���-u�][��Z�lr=��(�HQEQEQEWk������-2��n�qU�|>������q����(��Eo�����7�Q�QE�A�Q@Q@Q@Q@������q���/�J�������/��������^�foK�QE�lQEQEQE��%(�@I�2��o�����uU��2��o�����uU�?�Dz0�PQE�AEPEPEPEP�x���������8��b���(X�lT���fR8 �+�<q�I��g'�U��A����$��<)�jR$h����K���W?s��@�h���5�s+����w1�{��h��������l�KK��X\�R�zE]��PMWA����k$��d[y���>�/b=+B�
(��
(��
(��
(��
(��
�M;�NcV��n/�+���4��9�[�����p�K��(��(��(��(��(��(��(��(��(��(��(��(��(��(���d�`����c�m�0s���Ul`X�z�+c�S�[>���g��wu�*z(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���-{�FMO������P��������}�����_sO�^��O�aEV�Q@Q@Q@s�*��_����������UGr*|'=EV�(QEQEQEQE������WvC�A��^�+�����]��������4W�\s��O�?�=L/��aEW��!EPEPEPA�E�y���#��v5�Z�������}{���u��QEeQ@Q@Q@{��������?�W�W����Mo������w�����F�~4z�Q_NwQ@Q@Q@Q@^(��OU��I?�^^�����_����A5��yG���x���@��+�<���(��(��(����S�����uo�Uj���!k��������c��QE2B�(��(��(��o���������v{��q5�|>������>���o�?�Q[���#��TvTQE~z�EPEPEPEP7���q�\_�A5���G��F��<.?����&��~��W������QEw�Q@Q@Q@(�IJ:�����������m]Ur����������m]U}��G��>QElPQEQEQEQE#�I8�4�k?|'��I�5�~���UY��L0d�<w�K�����V�u�]Rk�g�Kl��!L`pC`��*��>)�:�\�(�6�(����{M5'c{r@���>����R�����S����)y�$��+��8�w�	]I<
�����"�!s��o�3�Z���(��(��(��(��(��+�4��9�[�����p�K�4����5o������=.�(��(��(��(��(��(��(��(��(��(��(��(��(��(��!�lH����M��6#�@Y2����LjB��t��72YB��KF	`=*j(��(��(��(��(�����+�jVv�v6�+qH�W`A
���J��ilL����@��C����@{/��������e�^�?�0����o��C����@{/��������e�^��?p��=����^Z���������W��=���z?�0���@��+�?�yj������G�/-_���_�������m����������������������C�=�o����5x��J���`�e���F,6�n�J���Z��
�#%%tQEjPQEQEQEQE�Z������}�����W���5?����C5B����������(�
(��
(��
(��
�|U�.��?�]s�*��_�����T�Nz�(��P��(��(��(��(o�����]������x�M�^P�(��#���N��zm~E�?����������QE|	�QEQEQE�PzP�k��0_�cY������}�]�g��O�^�Z�QVPQEQEQEW����Mo������x�{��������?�W~_����kG�G��E��pQEQEQEQE�����_����A5���(��OU��I?�^_G�^���|q�
(����
(��
(��
(��
JZJ��?�]�V�uV�j��������n�9�ES$(��(��(��+��|��_���r
���5��m��G�~�9�g�<��o�?�Q[���#��TvTQE~z�EPEPEPEP7���q�\_�A5���G��F��<.?����&��~��W������QEw�Q@Q@Q@(�IJ:�����������m]Ur����������m]U}��G��>QElPQEQEQEW
��R��~j��C�";DppW��P��k��O�O���c����*������d�����?8��
_xoM���k�i���%T���1�I��>)x^�����)��Ki.m.�����z������_�7�w�O�Q]�c�-����>-��C<m�,��0Fv����������4MRrL�v1J���Td�u�X~
�����t��H����8���*1��q[�Q\���e�h��:V��=C^��9�'X��d�4��������
��_����>�/�6l�����C�olJ���_��9��m��I�-KP�������N���#'��������k-��������c+\�����7(oj�
+��������~��K4�[Hd���U��sV��?V�������;o�H!��k��?0�I6�����=���g����r}gWfC�T�e�s��������������|}����,i~nRo$���PS��N��u�O������Rq��~c����RI}k�����e�E�k>��|9)S���#�1�G����@�^i���j��-����z5��7��]ZH��2	#�NC)~�:w�����b�_�8P��EQEQEQEQEQEQEQEQEQEQEQEQEQER>v�q�-5�P��8�:o�1�'�	�H�!�?O�M5���������M =C�t����?��Q����T����&�������������X�������`QQ��_��x��O�f��&�������������X�������`QG��������37��4�?������G�&�����������>������mS����	���= ��?�z?�4�?������XQ��_��x{j�����M5���������|H���mN�K�B���x�]%q�6���O����mG��>Y���TjNN���Q]��EPEPEPEPg���JV��\���f����>��J������,��u�Yg�_���>��+�7
(��
(��
(��
(��<^���S����3T*���#&��_r��f�W�����S��QE�EPEPEP\�������+��w�_�����Q���	�QE��QEQEQEQEmxGg�%�f?�n����8W�xA�|Qk�g;��8;Mzm~E�?����������QE|	�QEQEQE�PzP�k��0_�cY������}�]�g��O�^�Z�QVPQEQEQEW����Mo������x�{��������?�W~_����kG�G��E��pQEQEQEQE�����_����A5���(��OU��I?�^_G�^���|q�
(����
(��
(��
(��
JZJ��?�]�V�uV�j��������n�9�ES$(��(��(��+��}�m�O3)�����*�_���/�>P���_7��'��������*;:(��=@��(��(��(��(������/��������^�{�����^\�t}+�����u
(�����(��(��(��u��h�?�_�M�o���6���_�_�M�o���6���������F

(��((��(��(��(��3Ey����A���/����ZX�-��}���$i��2O�4<�����u��vuXE���G� h�}�'<|�}(�h�?kx���6�?���(�v*3�����^��0��f�<�?���Q�^PK��T�9^J��7��*[_�	yi��
y�Al�xI �
9s����!�����=��xs���K$�yt�A�G ov#�'�za@�O�?j-)�R�C�XT�X�����
�|��G��Zk���I ��|E�5=�z��}>mI4���Q��]��v����	?�cx�T��������[����{���[4����<�94�'���Y\s���S�H��V���h��9�;�����ua"�=�H�i>)�:�����_��7�����ym#����8_R==����.>-�������A��o�^_Z4	m
6�2z�@���@|F��[�����4W��S���2�����g�b�+���"�p
�����p����S�z����*��0y���n����UN1�@\b�����������|=�E��aa�k�F�;$��Y��8��P_��&���+�;o�lc����+��Ue���{IQL/c2�l5��������X4
���V	l�������g��`k?��F�����Y-��d���mw26��?Z�~�Ky�_��NK2���}FQ�?
��-[Q��h�F}'B����������nw�d��z/�|:<'��+C�[�;/F~�G��k�����5o������:�\��[w�S�����.���9e���F<~�V��Spc6�8�7.?,���^;����I�K/����n'1;5����)e�~�f�������d�v�l����\P��h�i#3��_���@���YB�H��I�{u�3�.<�l�3�3r�������+��4�Z�EQ�9e��i�vW-i"�����Y��+��,��@��&����A��B�-$,I7.G�Z�EV{��P���Xd����N��<�0U����.p?Z�E@'��yf��gf����$w9`��& �^O�Z�EV[��7f��Yq�,�o���O���$��cf��=z���@��u�m$fl�C.W����eei#�$2���j��Qq��g)�y��\g4�<�1V��TgYpZ�EVK��\���Td��{u�\Naf6�7.O�Z�EV7�U�����M�������U
����H�)��Vh�
�<�0U����.p?ZQ<���6�8�7.>��jz(�w30kIH%��n��>g�B��!�,�7�V���-�����?|����7��.<�1�q��L��jJ>���_�<)5}������V|��_�����.=������h�/��au�\Z0=i(��?������z��zJ(�����]�����C�����&eS������]\w�������7��]i/h������\�/o
������;�#a~�Q�x|��l�n�������Ez����r��������Wr���(�X6���i e��*�s.�N��m��u|�ml�3�P���[���3V��q�*
)]v����M�����B0�=zP����E�eH9p���*���`,xL�
�a@��c`�N��-�gp�� S�b��{�*�s.�s��T��,$YU�0A#a�'�z����_��fq�n\}q���	���i���o�k�����/����	].&g`�� �K/?�"\N��5��Tp����j������?�_��n����8����q0������_���@���vm���('�)��J��.gd���e�����P<�9F�uN~r�������u���re���Z�EV[��L��@��&��~�����d�ql����Y��<YU�E�F~�)�q��8�*�T�*��A��k��2j��/��j�}�8�E�c�&�3�~���y��w42 PD����2��W�����4�H'����M�y��}�er���]v$�b��py�X��3*���y�QG+���a������w4Q�*����(������4���<�+����Y�\����*�=�r������UF.��DT�����}����"M��}1����I"��H�@V(�q����V�����G?2�Y�����b�� )F$�C�'���/�����g�����q��E���.e��%�	����H��`G�;R������N��#a=�V�9_����v-[��tE
�����1��3���U@��l7���E����C�]�Q�@���E����wQ�MKxZ���������d�8�W��W��2�t~�(��I�$vJ����������>.<Qk��g=�����_������}}���=<3^�ap=h������?���:.���F=�(��?��������Q�zJ(�_����]v��ZJ(�����������`s�)(=(������������16�~�u?���*�?�1T���]F��������:�������}{t��Zt:��� ������������J�/cel��6���V����X�<>qO�G�n|����t�	oH����R6�F�}8��Q��b8!ub���A )V;���F���Ta��+kg��b����`'x!X�-�n�@���v�{xU�-�n��#|����E>e��0AyR	2ll��5�?I������X�a�?��[����c�����u��Aj��4����kK�G�=���V�G,� 2����9��f���q��\�5=��q�)��m�&q�n\~Y�$w6���&�$e��>�j�Yn'1;I�R���h�D�N��I�v6n\���*�Y�'X���Ff�e����\L��m$p@$�^=���(<����S8�7.>��iy�R�k"������b������]TKn��I�3�'�����xr�{��?��������	����St��S�������4�G�����"�������A��E{<���yw]��!BL��H4_/w����psL��W�.�h����y�<R��0e �8<TtQ���Au�����.1��4�4,�������9_�.�X����px�<��|��j�����2��W�.�~�o
�\:�F�e9�#ds��qU���F@��8f��6�S�F��!k��������.��_�r9+�Z6��uA}B	2ll)��3@��3����ca��1��E>W��� �]�2�v�������c�z�
H��hK��h�8B���c��E���.e��m��`�����[[?L��-�EB��HY� #
���Uh������v-=�����N�����^(�4h������663��f��G+���s.�����tk��W\�a��f���7��b���?�p��|>-�_�
������w���k7���GNK�-;��n���%�o2�_���J����`z�QG4�~?�]�������9�����0��/n�v��RQG2�_�������`z�QG4�~?�]���6S�����)�4�����%Uk�l7����������/��������^�J�O����5������o��=�@�1�=x��#o�����(�g��b�������,!��6c���?LR��*P-�n���6�S�W��e��2�������_n��[�fd7��"M��}:f��O�v��w
{8V(�7��� ��fk���8B���w�(��vc_g�>�|g����g�<��K���P�6��*�(�J��G�9v�����FfX��HO��g��]/�'�K��M���r��q\��/�&�7�p?�WW__G�Q�G����q0������_��K��n�Ip��/�}:��+R�yE�Al�2?xq�g4,�4���q!e��sS�@�������6�FY~c�������m$�n\����@
���Vt(H�RzS���
(��<�������c�xm�n���w�m��X���+�a�\�4/��
G����]K?��K��:�,�����p�s����������I��\�@��q,3[���w��I,�j��-�2xsF�T���M���_2�m�G���Y���`d� ����]������C�p�1��j�
�^S��x�G��e���[I���j���!]��������c?��L}:Z(�Q@��(�PF(��
�M;�NcV��n/�+���4��9�[�����p�K��(��(��(����<Y��W�m��$r�b@&n	#����G�vL�+�����o�X����������X�����������#��e��?������o�X����������X����������b?�����?������o�X����������X����������b?�����?������o�X����������X����������b?�����?������o�X����������X����������b?�����?�������7��&��Eeeq#�.v+B�'�=t�������!8�^.�ETQEQEQE������i��#c�����>8�?:e>S����?e~[W���<	n��(��QEQEQE�x��?�?���U���o�����M��WN�����s4QEz�HQEQEQEQE��'������9��Y����O���R����7��5�]}Y���Ge�(���
��(��(��(��(����d����_��
������������4��|��6QEh@QEQEQEW;���u���J����W������Tw"��s�QElr�Q@Q@Q@Q@~|Qk���br:��"�6����[��b�x���+�E~E�?����������QE|	�QEQEQE�PzP�k��0_�cY������}�]�g��O�^�Z�QVPQEQEQEW����Mo������x�{��������?�W~_����kG�G��E��pQEQEQEQE�����_����A5���(��OU��I?�^_G�^���|q�
(����
(��
(��
(��
JZJ��?�]�V�uV�j��������n�9�ES$(��(��(��+��|��97'��5��m��w~��2�on����G��+|���t�����(��P(��(��(��(�����<.?����&��~��W�������q���/�J�p{3z]B�(��`��(��(��(�i)GZ�O����}���
���W����}���
������(�#�����(��
(��
(��
(��
(��<����|+�
�g��(����ii�i�&�Wa��g����������~1�n4/xS�f�����T�q��3t����5��_k:~���n�k}�Ad��s^HVh�q�9f8�q\I���_��O
;A}"�%���]����������Pgw�����d�=��.#��i�\xP��[����f�v����z}����y-�<�fI����	����@Q@Q@Q@Q@Q@y���'1������u��w�����b�_�8P��EQEQEW�|Q��������Cz���o�?�4[�����o^�W���ga����+���
(��
(��
(��
(��5�-����&�������I_8\����^��G�����������q�_�W�\u��x�s�����:xu����h�3�������4���g���'��k���2k�j���5����������?�?m�?�G�-�~���&�^��&��j����j�����h�3�������4���g���'��k���2h������� ����������?�?m�?�G�-�~���&�^��&��j����j�����h�3�������4��C��N5�\���&�_��R7
�w<�(������� �������q$����;��m62�5)�v��LqN����^��p��+1Q@Q@Q@q�6���O����v5�x��?�?���U���*4���Q^��QEQEQEQEv	���i���o�k�����?���?������}_E�����C�
(��Cp��(��(��(��(����5?����C5B����2j��/��j�}�?�z%?��QZQEQEQE����]���*�|U�.��?�U�����Q[�EPEPEPEP�����1��8�i�zmy��JZy��-���W�W�\s��O��=L/��aEW��!EPEPEPA�E�y���#��v5�Z�������}{���u��QEeQ@Q@Q@{��������?�W�W����Mo������w�����F�~4z�Q_NwQ@Q@Q@Q@^(��OU��I?�^^�����_����A5��yG���x���@��+�<���(��(��(����S�����uo�Uj���!k��������c��QE2B�(��(��(��_���o�?>Pc=��\Uv��m�y�C�s�k����D��_�R:p��GgEW��QEQEQEQEC{�����^\�t}+�o��������k�����z8=��.�EWy�QEQEQER�����}'���I���\����W+���I���\����W�P�}���AEV�Q@Q@Q@Q@7�"������4;��_EKY'3��m�s��9��j>7���e�~�����	,m�(�m^6�Wf����[�����m���5�[=GM��X��� �c���G5��=<Y��@�
oh�x{�zF�u �k+8������
mU}>��T����������M�����@Q@Q@Q@Q@Q@y���'1������u��w�����b�_�8P��EQEQEW�|Q��������Cz���o�?�4[�����o^�W���ga����+���
(��
(��
(��
(��3����}�B��e��������6���=]��(�2
(��
(��
(��
r:�gz�i�H�H9���=��S�d���ZD$��$�C�Z�j��Iz��{�QY(��(��(��+����Z�&�����;����i�\��B��/�Q�?��h�������(��(��(��(��O�%+O��s��_E���	���i���o�k���,��/����QE��Q@Q@Q@Q@����������^���S����3T+�i���)�l(������(��(��(�w�_������W;���u���J��EO�������
(��
(��
(��
(��6� �|Qk�';���y�M�2�ya��]��!����y�M��������Vz�_����(��:B�(��(��(����J��w�F���k>�5��/������i���[ ��*�
(��
(��
(��
�?������X���c�����u��Aj����#���h�h��(����
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v���w���(wc��W]����]����C�=���o�?�Q[���#��TvTQE~z�EPEPEPEP7���q�\_�A5���G��F��<.?����&��~��W������QEw�Q@Q@Q@(�IJ:�����������m]Ur����������m]U}��G��>QElPQEQEQEQE���T�>�}��@�Y�/WZ�������g���Rk;K������i�����t-2���8�=z
e����?h	�<E,v�j�<p��S�������������3�������A}y����A��ify&�(9�����=�E{	4;)4x����F�H�"�de@���W����s�|>����E���Q�P�FEt4Q\��|u���x�R���0�Vf���S�!�M�d���Ym����XZ3�z7��Er�-���x*���\��K|����1m����U=���|G���i�R�L��y�t�'
X��p8�;J*���i��e�������<���=���s���%�mU4�7Qe��f�`xL��7��������VzN�=��s����i[
����to��5ht�=Q�{��nn-��'>��4��Q\��gk%���1)y$����d�OA\E����j��C��������D�F�0$#mw���w�����b�_�8W���i��s���q��@�EPEPEP^M�G�F���_�
��k��(���o�^k���zy_������2�(��>x(��(��(��(�����q�_�W]��� [����B����w
(���(��(��(��)��1v���}i����F�!��eL�]����t�)i�� O����W���g��
(��QEQEQEW�o�����M��Wc\w�������7��]8_��J��E��!EPEPEPEPg���JV��\���f����>��J������,��u�Yg�_���>��+�7
(��
(��
(��
(��<^���S����3T*���#&��_r��f�W�����S��QE�EPEPEP\�������+��w�_�����Q���	�QE��QEQEQEQEm�?�%�1��g�i�zh�2�z�x��km�bx�6�+�E~E�?����������QE|	�QEQEQE�PzP�k��0_�cY������}�]�g��O�^�Z�QVPQEQEQEW����Mo������x�{��������?�W~_����kG�G��E��pQEQEQEQE�����_����A5���(��OU��I?�^_G�^���|q�
(����
(��
(��
(��
JZJ��?�]�V�uV�j��������n�9�ES$(��(��(��+��}�����r}s�q5�|>V���w���w����(��Eo�����/�Q�QE�A�Q@Q@Q@Q@������q���/�J�������/��������^�foK�QE�lQEQEQE��%(�@I�2��o�����uU��2��o�����uU�?�Dz0�PQE�AEPEPEPEP�x��~�����<?t�S�<w��m���n���x/���x���iwR����/��~IS�[�*����5j���R�u��R���d��b�KU�m|o�����������������/���@QY>����
������5(m�.g[��NOZ��
�Ze���Oyeoq,9���%fL��#����� ���@?�=q��z>��iZ*����i`��#73�{����k��x�E��u�{�Z�	�u����UiCm
T�OLu�
_�PkP�N�n���7����v��m���U�aG^���m��W~&���&��
��#6��mh�=����j�5x{J�J�u�;+��L���#,��'��==+�>$kzo��m����QjZ�����Mh�b���x��=����5��0x�7�I`�=���+#F�P����uc���/���uX��[�.mgQ�C�)�s��T�xs������O�����ER|��}�5[����xN��^�-�MO]��8���K�0������;����$��z���(���O���V��]���&�1��|"���q������Ed�^�����	x���_��J�B�7��~t|T����|7��|9��j����kemi �������^���#]���
xZ++��{8.X��e��Ew�7���}��T��$������F"dRP�B�;�W�/t��*XBg����\�G�X�$��9?AZ�/����~^^�Z�����h��ZE i�Yh]��A<�(k�.�>������nd��|�v9,Q�d�����>(��'�Ej7� �[;y|?H�6|���aA=�����?�~hzE�����j�u��a�"��8g�����[��G
��h&�y�=En�I|�a�����/m���3$m����(�w���	(%f��"�����������(������y����8=h}B�#IP@J�qVh�
�_�D�PG�k��')���2�v����z�y7��-����7�K,���8�{N=��P1\�4�o*��,p9�TW�Z]��'����L�P�# dP-�.P'�H��������w������X*T��8�X$d,*;����K���=���~a_��iZ	P���EEE���w�)��YT�-�dsG�/��g���#�EE���{������ZO6�qQ�|/��E�]����X#c�pG^��k��$�E���q����������_��Y-k��0���(}>�8�	�0��w��U�]��&W�b��e�L�$L��~a���m2�gXZ$`J����T��O����������-
2��hD��1]��X���^E��6��p��uR�->�����o��f=>�Xh�&5�[#�u�Hl.V��&#��;�:Uz(����Ex������u�*�$%C��wI���t����)!ein���t�p��s���O���w������UUy8P2O^)psL�h�v}��n=1�:����{G�����{n����IEgxv�h.(�i(��������4`��Qz}���4�RQE�����Apq�����	Z���2�
g#�nu��x��?�?���U��p��G���4�nc	�.���!�$(���`���>I�
��������`zW�x�����B��]<��K�7�3��Gau(s%��������zQ��E������:��5��b& 	������H��a�;��1F){��B��QBT�m^G'���.�e��!����?���F�;������atf1O��q\��BX]H��	-��p��u[��J/������d������Y��z�c���~�nn<�3��������q�+O��s��_FW�e��.����������GeIAe�����Z�n�(*�,px�4W�nV����L�h�npz����D�4�#�i��f��%��[<�@������k�u���HHp{��P{n��V@d������[J���]���U�(���H%����ua��6�Z�Ex&���I�]�n� �\�TV	K*����������������z|��^����I��2O&O+���c9�
���`��GE_������IZ�UeR�8�h�<��l;��2**(����C���}���R�Br:��	[;P���QQE������o��$�c����{����������}�����7!�"�<Og<�Y��������J��{�_�����QS����*8���}�@�o
��@|����?:X���d�8�%� 8�8?�T�������	�xv������������w1��'�.~���(�8��#�V��K���W�o��-I�]��$%Q��s���d�/"�4xin���QE�����;�����t��p 00����O���ow�`&H�,������E��}��B�����-G��Ld�X��[�N��au%��"& 	-���V��K���W�o��:?	����\H��`�`�����c"���"3��O3��?��q^�+�^7��S������OO
���B����J+�oO�����F
%^���4�F�Qxv��\F
%^�g���
����������������������1����������m�����c]�P_�cT1^�>^E�C�Z��%��A���B�#�V��IR6��|������`zV��o�����]�>I�������,.���!%� 0��?�V������������neFh�,�H'#�:��w��#�c;�:Tb������B������!�s��+�]F��	
!���O�U�(���x������l.��	�K��~u�?\i����~����y������c�����u��Aj��r�b6���i[�����j��:�^"����0���00{�����;����<�&d�6��Gm.����K7�*�Yug��YAD����h����|�4l��8=j�Y�d��_%N4�_�D��(�#��5b������ ����nZD���SH�r0{u�P�.����������'#��Z�3o(�!O����^�����_����A5��9O7����.cnx��[�\�O���B�����v�G��}��'������E�FB�r�<�Pa�G��z�5{����=�V�E�c'�A��X)N[������O���=����K�K�f���E�VfP�+�dqQQE������o��%X%pJ�ppy�L�/0/���5Z]���=����]%���	��
��O���e�M�S#mQ�r:5?�]�V�uV�Jv����r7����-6�gXL�`YWp�����4���e�8���S����B�����-��^J�,ph�k�����c��������-���z�E�������X6Kk���;�:S��n�T2BT;_�rON�V�-.����;_qi�����'���!��??�o>��y'���n����U��O����������-&�y$�D����p�8��]�m�8����S�#9��_��6��<���������k�d���O�)z�n_h����0q�0q�J+�����������0sIE�����Aph�4�Qz}���4��IE�g���
����/����]�5��I���^k���D�����dq���&��<.?����&��@�>����,��zV������p���[���"��5���	�������������]	���8D��a�;��Iau ����r:��0=(�{_phY6K2�a!���������O����G����`zS�{_phYK��!%�;Xdp:D���&�"%�NG����zQ�E�����Bsgp-��b>V3�#�+�����BUX��#�zw���������H|:�-�i���x:����_�o$���,6����w����}���
������(�#������[$I#J>v�qK%��[w��x<��EjQ^�����B@������y�+&d��=���Ph�iC�����<
P�h�E�R3Vh��"�����N��(��������n�m?Rm6y���,��?Z�|U��.�����u��]���H��3��
y
*�����m�O�����-K,�:����u�`q��k'��~����C�5H�k:7�f#�2�[��W��}]���=R��`��O���"gNp���]���=��Z�[�B8Un.U6�_���(��2��h~%HS�M���bP�6��=3�Yv��ks���t�����"Z((��q��h��c���6�ni:�Q�76��L�N��7��~&�C����o�-�X�}p+N�����w��$���E�==k/H������D�,,%���on����(��KG"����2=��}7�������-:������T~}���(`�+��^��N�g�i�_gw�#�E|���E��w�����b�_�8W���i��s���q��@�EPEPEP^M�G�F���_�
��k��(���o�^k���zy_������2�(��>x(��(��(��(�����q�_�W]��� [����B����w
(���(��(��(��)T��[�#4��8�H �=i
n{$d��
T�lS��E$$A��-5U�$�Y����(��QEQEQE�x��?�?���U���o�����M��WN�����s4QEz�HQEQEQEQE��'������9��Y����O���R����7��5�]}Y���Ge�(���
��(��(��(��(����d����_��
������������4��|��6QEh@QEQEQEW;���u���J����W������Tw"��s�QElr�Q@Q@Q@Q@~e_Z���8�;Mzmy�����-p��v�O5�����S���S�?�QE�'HQEQEQEPzQA�@q����}�]�g����#��v5�^�?�zkdQEYAEPEPEP^��#�=5���������?��[�����-]���~����EW���EPEPEPEPW�?�S���O���W����E=W��$��MxE}Q�9z�&e���(����((��(��(��))i(����-u�][��Z����Z����������{�QL���(��(��(����.�����9�n{�]���>U��8��wn�W��G��+|���t�����(��P(��(��(��(�����<.?����&��~��W�������q���/�J�p{3z]B�(��`��(��(��(�i)GZ�O����}���
���W����}���
������(�#�����(��
(��
(��
(��
(��W1�o�>�j��h�����5������t�P~������I��Y ��a��m�Tt��kB�(��(��(��(��(��(�4����5o������.��N������[��G
��(��
(��
(��
�o�?�4[�����o^�^M�G�F���_�
�����>L��?�q�QE}Q��EPEPEPEPv�� [����B�������}�B�������QEfAEPEPEPNL������M�&|�����!��eL�]�
��;R�&B.�����)k����%��u�QEd ��(��(��(�;����i�\��B��������ro�
�p��F��#���+�:B�(��(��(��(���?���?������}_:|'������9��Y��������_�;(|!EW�nQEQEQEQEx��#&��_r��f�U�{�FMO������P����/C������+B�(��(��(�������WE\�������*��>���+c�(��(��(��(����ya��]�vOA�����/�>(��@����m5���"������Y�a��
(���
(��
(��
(��
J(=(�5��/��������`���������/C�l��(�((��(��(��+������]a��Z�r���G�zk�����/�x���5��������s�(��(��(��(����G��z��zI��������������	����?�/S���8�Q^��Q@Q@Q@%-%p��������:�V�?�]�V�uV�[�p��)�QEQEQE�|>-���
�Ns�y�&�o�������)�9�5�|Q�"��/�)8_�����+���
(��
(��
(��
(��!��������	�._�>��7���q�\_�A5���G�����P��+��(��(��(��)GZJQ��>��e�$�F���Cj����e�$�F���Cj���(
>��a����+b��(��(��(��(��(��(��(��(��(��(��(���N������[��G
���M;�NcV��n/�(����(��(��+��(���o�^k���z�y7��-����7�O+�x�3�0��QE�G�Q@Q@Q@Q@����n>���
�k���q�_�WZ�c���EU�Q@Q@Q@9dP2@��6�@.��3Hks��mE� ���Zl`�/*�lS�����%��u�QEd ��(��(��(�;����i�\��B��������ro�
�p��F��#���+�:B�(��(��(��(���?���?������}_:|'������9��Y��������_�;(|!EW�nQEQEQEQEx��#&��_r��f�U�{�FMO������P����/C������+B�(��(��(�������WE\�������*��>���+c�(��(��(��(����z��E����)��4W�xAU�Qk�����5�����S���S�?�QE�'HQEQEQEPzQA�@q����}�]�g����#��v5�^�?�zkdQEYAEPEPEP^��#�=5���������?��[�����-]���~����EW���EPEPEPEPW�?�S���O���W����E=W��$��MxE}Q�9z�&e���(����((��(��(��))i(����-u�][��Z����Z����������{�QL���(��(��(�����.��nA��~k������������n{W��G��+|���t������(��P(��(��(��(�����<.?����&��~��W�������q���/�J�p{3z]B�(��`��(��(��(�i)GZ�O����}���
���W����}���
������(�#�����(��
(��
(��
(��
(��
(��
(��
+?V���6��Z��A*n&T����_���|K������
F�P��CIn�>�"�5(����e���y��Cim��;�U�Mg�^/�����@�,���[y��u��l�P�]����sy<v�D��Y\*��$��������hZ������PN���n�L�X�����8�K3��P:�OAXZW��-�jM��:��wv3��������PAE#0U,�I'�s��?������e�-6{��D	r�����@y���'1������u��w�����b�_�8P��EQEQEW�|Q��������Cz���o�?�4[�����o^�W���ga����+���
(��
(��
(��
(��3����}�B��e��������6���=]��(�2
(��
(��
(��
U�a�����%9N3�!��d�i�v}��n=1�:��V��������^��^�EVB
(��
(��
(��
��m�������*�k����Z�&�����TiO�9�(��s�(��(��(��(������J������,��u����)Z�s��E��.��,���#���QEz��EPEPEPEP�k��2j��/��j�_���d����_��
���>J
(�� (��(��(��+��W������tU����]���;�S�9�(��9B�(��(��(��(�
����-<���{i�zp�2���E�A;�O��^�+�.9�~������0��+�N���(��(��(������<�]��������
w�F���k>���:��(�����(��(��(����G�zk�����+������]a��Z�������#Z?=n�(��;��(��(��(��(����������	��w�G��z��zI������<��r�<L����QE�PQEQEQERR�P���Z�������kS�����uo�Uku���
(��!EPEPEP]�������������]���]�`�(s�;�_7��(��������*;*(��=@��(��(��(��(������/��������^�{�����^\�t}+�����u
(�����(��(��(��u��h�?�_�M�o���6���_�_�M�o���6���������F

(��((��(��(��(��(��(����|��]Z�k�U��%�� ���n�x�s���/�(�x�Q��5��UQ�����"&7��zd����4��/��u���?v��8c�w�Ox[R_;I���S�����d�;���}j�:U��o|7�=��=J+Y��b1<2d`�q����xs�����~�
WM��N�
��	������|��1xo�VN�����49)d���s����QS�O�~�U�c�_M-��A�&H�Bn
E����w��m���[i���u��k�Y�(v���u��/����-x��G��$��xX���,{O�S|x������Vr$���u���g�������� ��&�a��A+�o�mB1s���*�����,xF��gu}�i��u���ucqk��(G�$�����mO���z��l�?���������>�'�+g�'�,�>jb+���S�[Y�n��F6��4��/�7���}����x�K[i8!d�L�C[^2�e���U��v��Z�Kah��\�Yc�5�x�$�y�5�|I�.�_���D�'�'���@��Q�?�5�x���U���OZ��-�,\[0p|��HU�?�>�c���?�~hz��o���	\�gRP���k�:�����:������O��Qd��bVo78���]?��}�O����1��wF�v/��7W3���������ak}xvX�aY[��@`y�<���������iz���+l-op��q�O<�*��'�%B����?�U��4�.2�f�gf������`������"y�$�v����n!|��6�2p��@�����2���-�+�� ����h����������������n�1�J
�
��4a[�������g���n���m�eUhc*�Q�@\���0H��Ey_��_�����($s����� �Q�Tq^W�@��<!NZ) w���,��XV�p���58�����)Ln�#�!w#��L���.�9<W�{�������ll�b�-�ccdu��9����c�;�'�h���F�8F8����#��F#�	g�=y�@w�L���.����Pcq�����{m�����(29�]�=M�{���
0'��Z<�������G$�����I�>��l��h���(kP��E��N���B�~��=m�pJA+8$!85�kr��=�$����1�
���t$�(c���Z���9o���s�!�����q�����v�@�b�=9��"�������?*�f�#M!A�)s��j�x���n�x%R�

��A��0J���r
�w*^yX������ng.�)`0y��h���y���2J\��r��mn�H%b���H.g\O(b0[y�Z���U,r�9>�{��y��N��H$d�$u����"�L2���?:�eB�4��9P��j���4�^1�y��G�/tV��@�*�8�4�mr�F
����T���Mk�����N@.N
(��i��������h���.{
n*��� =8�`�SP�����3�\S�_��y=��}���\0h�4V~���Q��Fh�����
`�Q�3G����4f�s�4W�X��Y�F#�a�;��]�q�5w�`;���=�u��7'�F���s��ueV�@��AC����<��y2o�v�9��(3������K��<�������n�s��z������vb�/P�R-������N��,&�3u!�M<���H��N��=������:�
q��`�c�H�b�� �Q�%p��q�R��d���P	c����[������e��6��a�1��M#O3ct�6���'�<���,:6��O�
���(!�p+���*����a���0N
'�0r�i7���H�g�Y�� ,rHr3G����d��a+#,a&���<f�����<�57�w�U����f��a32��' �,�������<�����3���e��.����������FH 0��\����FB�!��o
�V$�HQ��R�`7Ps^���i�f�:=�����70f�+t%�
g�f�&=�������T)�2��5�	��F��F\r(3�$������*����m�*8����h_��F:A<-&���t0�s��4m�d���J��$���4-�+��F7(�P.`(XMU�w�
O���?��px�iE�!J�c
z������'��r�������x�R(����9Q��5@F�d#�_���Z�BT}�^������f�5�����>R|��66��[o�8���d�}E�n����<P]��f#����{���
0'��Z<�����LR�K�#�=(�vw�}sG���[�@F��1������JO1�$;d�94	g�=ph���
�WpF��8����U���o`�C6=3Av+��c�4{���cq�����\��!���',w�B�����.�9��J�h�#��C��Q����0>�pd1�%�J�9��)V����`��N$��'�g����#����4-���2�*��A?Z��9��[i�K$2�A!	�g���|�<�g~��}iV�tR�4��9���y^_�'��l�q�����Z�tP����CZ�+*��1�����
q;�W�FQ�$
�vef�RT�Is���|~�}�����#!v��[���A)u���~}�0?�.�0y�=3@��9q<���94{��y���>��J�NN��[�����Vq���3�b����[FO��,~Z� O��8�����~��?Z��y<�� T����J�o�p�$��J��6U��
���I����x��^�����=<5���>�`�Q�Fk�}�3�@��4f�s�4
1IK�=�0�1�F(��4{�a�b��)sI���{�a��z���"�!O9�`��P0J#�R��S�����J���,����q�U4�=�W)����{t�y���d
o2c|2.���#&���)�@��94�<���H�NF\��f���%�Ei��A~�>��L����q����������#�I����&�cv��}h����Hu!�M�h���20�'y2�~g�'����q�������E���f���������m�q�R�CAZ	�C42=	B���VP��((F~��e
��Tt�
<�Ai�%NA.x�����y��0�����rG��{�{k]l\�$���'��-�^C��\9�M�`6��>����\Z�f�y����>o�oZ��r�b6���i[���0(R�F���Pn!W��c�K�������Q���>��x��}1�x��X��w�T-���F�FN
QBO0D����F:E��3�(����������^�8���0l��G�8��c4xB���������g���n���n`UVi�
�	q�
q
h���h6���hc!z��� �HH���>'�eM����?�q9U��0���"2y�$����3��,�X�1�B����$�M�MTE"H~�'
��	�
1��ll���r�$1��MT���$����&�4��������������1�<n[�����1@��p�q��c�����M ��p�3��{>������#��68]�[o�8�;�f�4om�w6=3��{�cq�����1� `OA�����.�2h29 �l��=(����}�����P#rH���)<�������(��$;z��h�������&���km��	g�=ph��v�m����x=���-�]J�F�@�V�8<��f��B���K((FO�Y�.&mJ�i
	[
\�s�U��w*ZyX������V���#��[� C���]�$}(�B�	K��]� }(73���.o9�@��H\O(r0[y�Z~�{����	X���������e�r�	��ngB�g�K�FO�"�N�Qf�)���
���6����<g~�����n�J��P��C<�/,�!������k�����NF\�=���n�	C7�R�'�G�n<�/��~3�a�=qA���Y�����\�}(�L�f�>]����8�����������(C�����"�~��1���0\�������r~��|?yZ;��������k�x���5�����������ge�1Fh�~����0}(��4�w�=����4f�s�4Q�3Fh�<�@�>�f��=�0���cp$���My�[��(�B��r>������	D����^`��M f$9��C�foJ����>vC#m88Bph�c�R�w8��Y�L��E�����H&�G��\&1�1��]�����&T��H�b��
o2�����A�VM�,�G��8��i�bM#9rph�C@6��a�1����y������r��<����,8
��Q����M�`���>����U�����!*p�!���,2�XdeHB@�g�Y�� ,r�9���UR�,�OP�h�C@�e��)6c;��~t&@�"�@�������5�cw~T�y�����\�R�CC�����|9�VV��w�8+��_J�>�����fq�x�k��l�/��%�P�$ya��;u��������G�9��c5���Dz0�P������n����k�S��w���Pm�*�U�
�
����m�*8�@g�I��P?�K�T�ZB�*T0����H_��F:,��$�(�"����4m�d���sB�h���x��[xW;a�n8Q��[���0���0h���2�y���T"�PJZ(��(��(����Y�]��Z��+Y.����I�#=Z���xO����K�I��Y�?o���A�y�z��xsI�>��v�c���\���A�����8���o^�C���]��YY�rHJ�(�
Z�����d�v�g���C*�[���\���xC����t�!R����W�������u�P{�]N�k-B�;�Y��������"�M�'��Z-KN�#[�N�ZY^U����!]��Eso$��H�7\�)��Q\e������z����'�#G��(�9��IQ�WoE��w�����b�_�8W���i��s���q��@�EPEPEP^M�G�F���_�
��k��(���o�^k���zy_������2�(��>x(��(��(��(�����q�_�W]��� [����B����w
(���(��(��(��)�	�@89>��r���$�R��T"�9 �S�ZDQ@9�'����W���g��
(��QEQEQEW�o�����M��Wc\w�������7��]8_��J��E��!EPEPEPEPg���JV��\���f����>��J������,��u�Yg�_���>��+�7
(��
(��
(��
(��<^���S����3T*���#&��_r��f�W�����S��QE�EPEPEP\�������+��w�_�����Q���	�QE��QEQEQEQEm�=X���km�bx�6�+�k�� ��Qk�#���x�M��������Vz�_����(��:B�(��(��(����J��w�F���k>�5��/������i���[ ��*�
(��
(��
(��
�?������X���c�����u��Aj����#���h�h��(����
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v��yW�����n;��q5�|>Q���Nr���5�|Q�"��/�)8_�����+���
(��
(��
(��
(��!��������	�._�>��7���q�\_�A5���G�����P��+��(��(��(��)GZJQ��>��e�$�F���Cj����e�$�F���Cj���(
>��a����+b��(��(��(��(��(��(��(��(��(��(��(���N������[��G
���M;�NcV��n/�(����(��(��+��(���o�^k���z�y7��-����7�O+�x�3�0��QE�G�Q@Q@Q@Q@����n>���
�k���q�_�WZ�c���EU�Q@Q@Q@*�p��s����Htg�@�����F�>������M����J�8��W�M_�K��^�EVB
(��
(��
(��
��m�������*�k����Z�&�����TiO�9�(��s�(��(��(��(������J������,��u����)Z�s��E��.��,���#���QEz��EPEPEPEP�k��2j��/��j�_���d����_��
���>J
(�� (��(��(��+��W������tU����]���;�S�9�(��9B�(��(��(��(�
��>(��1�[r�i�zp�2��"���x�w�<�8�M��������?Vz�_����(��:B�(��(��(����J��w�F���k>�5��/������i���[ ��*�
(��
(��
(��
�?������X���c�����u��Aj����#���h�h��(����
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v��m�<y�O����\Uv��6_�>|��;s��o�?�O[���#�
�TvtQE~z�EPEPEPEP7���q�\_�A5���G��F��<.?����&��~��W������QEw�Q@Q@Q@(�IJ:�����������m]Ur����������m]U}��G��>QElPQEQEQEQEQEQEQEQEQEQEQEW�i��s���q��^�^i���j��-����z]Q@Q@Q@y7��-����7�Y�&���#E��y�������&pf�8�(�������(��(��(��(;]��-���!\mvZ���n>���
�kXls��(��� ��(��(��(�!��q�p;�i��1v�r0)
n{*���2���-"Qw6G��-5U�$�Y����(��QEQEQE�x��?�?���U���o�����M��WN�����s4QEz�HQEQEQEQE��'������9��Y����O���R����7��5�]}Y���Ge�(���
��(��(��(��(����d����_��
������������4��|��6QEh@QEQEQEW;���u���J����W������Tw"��s�QElr�Q@Q@Q@Q@~m�(����a�o��^�+��Xx��`��OA��^�+�.9�~������0��+�N���(��(��(������<�]��������
w�F���k>���:��(�����(��(��(����G�zk�����+������]a��Z�������#Z?=n�(��;��(��(��(��(����������	��w�G��z��zI������<��r�<L����QE�PQEQEQERR�P���Z�������kS�����uo�Uku���
(��!EPEPEP]�������8����\Mv��yw�f��=������V��H����Q_��QEQEQEQE
��x\���Myr����Q��������	�._�>����f���Q]��EPEPEPJ:�R�����/�&�7�p?�WU\��/�&�7�p?�WU_aC�Q�G��Q[QEQEQEQEQEQEQEQ�3�q@Q�(���Ef�
�M;�NcV��n/�+���4��9�[�����p�K��(��(��(�&���#E��y�����5���h���5����<�������EW�<QEQEQEQEgk����/��+���]��-���!\mk
�z��QVdQEQEQE����x9���fE�$����=�2w�H��-"����	=���������=��QE���(��(��(�������ro�
���m�������*���S��f�(�\�
(��
(��
(��
(��;?���R����7��5�]|����JV��\���f�����?�������Q^��QEQEQEQE�Z������}�����W���5?����C5B����������(�
(��
(��
(��
�|U�.��?�]s�*��_�����T�Nz�(��P��(��(��(��(o����-v8l�uMzh�2�z��E����)��6�"������Y�a��
(���
(��
(��
(��
J(=(�5��/��������`���������/C�l��(�((��(��(��+������]a��Z�r���G�zk�����/�x���5��������s�(��(��(��(����G��z��zI��������������	����?�/S���8�Q^��Q@Q@Q@%-%p��������:�V�?�]�V�uV�[�p��)�QEQEQE�|>
�������s�k������*��nA��~k����E�_�R:p��GeEW��QEQEQEQEC{�����^\�t}+�o��������k�����z8=��.�EWy�QEQEQER�����}'���I���\����W+���I���\����W�P�}���AEV�Q@Q@Q@Q@Q@Q@Q@G��+;�S�6���A���2]8��{9�1������� ��/~on��J�V��#�]�9�]�����
lu�6��Z�hV���0N�Z�����<���	�A�j���0���D�������Nh{�v���(���t[�����Io.�8�-�`��8?�`���������W�]CPq]��eut�G<Np[������_���h���������#�M�����T���~�Z.k��x ^��`l�MY���N��B���.�������n��C�/���k������;U���Lt��;���;������s�^���|w����������s�Y�B�g?�t����9�5�d� '�3:��U���{�;�������5� ��s���,=��k�����'�u�\x�L��R\�v�-��]]#8O���W����w`�z^�dns�9��5��/a��q����O��z�H�$P�	���o�z�.��[$�G��0�Eq�w�����b�_�8V��+Il��r�W�|F%��Er��-�M�D�6�W������d��=�������zP�QX:��|=l��_��J����������N:V��Qpf�%'9�d;*�����X�&;�4����H�H�:	f!��e$���@h��bO'��f����N?J�4C,�&pD��=��y7��-����7�Q��de&Y���)��y�����%!���$|����,��XV�p���u8�J��,�J�����������< ��c���TT�f.jdc`��@��-�<��l����r**U���Ny@iR�Wjs����`�{��RI�f��1��?�+LX���Qyv.�TT�bX6���AG���mL��1�QyvG����������6��b����$T������j�c�x��En����8���k+laQF��h�������+�7�����+����P1�, 7�w�c+G�Z��%��2
�����Tg��(k�u��n
���������U����/�o�`��G�Q����!�;�q
�>�����Z=��U��^8!6yhAn}�
��_#��n1�D7�]h��
��W�P�c�H�5fK��QLV�im�A8���7���#�n�������������N@P9��H�R�m�6�q�S��o�����{G�W����%��~Tg�Vv��Z	E.x��Q���E���
��?O����������QK���Fx��Qhw�A+����Z�&������G��2�Y�T#al��������Q��SJv�9\������#��	 , ���k��o�v���6�^���=��
��5ao]ew@K� �A���0����wD>��Z=�B�h�N�N��%pF���������c�
��+G�hA�3V$�y
cdm��}hk�e��*B0~��;G�hW�����O*��y#o�B^���%�HhA��E��4:����R����7��5�e|���S'�
�R�6�P���q�_A}�~��y��9����J��������>z*�Y�n�$��a�$�%�$n�Y�q�L���=��7,�U���I����g�9��k$h�3,�&pD��=��ZK%�fe�mP�,�g��i�h�p%2JA�$!*���m'2�&$��!+��6;%�8�w���Rq���4Ue�U���g!�$�NG����'�#�g�l��9������5?����C5B�u�0��RU
���e�q��QY��P��A����������\�Q�S�������3�g���,�A�D���E�r:*V������=�Q�6��S8�6~T^]���EEJ&`���=�P��Ny@h�������JD{6�=J��tI�f���Q�����Yw#�{�_������4������s�'�d��@���1�N��.W���<���h_�.�M�J�����������ICnL� }j���s�=��U���(�V�1'-
�3�i>��e�<�6�������E��V�r�fK��%C�
AaPN=M,��I$na�3��Gz/.�h�*�V����M���.��J���}�[�w�CnK�0�Q�A������Z*�w�Dd"s�������{R%����ER�b���E��V�sK�
���]��G8��k�Ey��/w��pK�a
����z>s��W��og���������(�=�R�������_hw����?O����������QK�1F~���p�J)s�����������PzR�����=?*-��hy����������Z�������s
�-��Pk�kq	��p�����j<�^�Z��3Fj���&���Ab?_Z��T��)��Gz���=
��5c���|�3�n<�����/dI�P�@��Z=�B�h��<j�#��$��F})��o'��n1���>����h�N�o$Ap1������^GF1@
��������4+����G�zk�����
��a/�@#���Y�*�n���Bc>l?��>�zWn/�F���kK�G��U��dTY���6�F~����W�Hd���=����z*�U����I�v����;%��%��R>iI����,QU��'A,�>2L���=��y>_�>7n��s��P�*��#��e��D��=�d�YI�a���Fq@(�
���;��9��0���R%���A$���e$s�@�Q�"���^��&�"����m�]U��s�I8�B���Z�#3
�21�b��)�����.cnx����L��m�'���fR�*�����yv<�G� ����s�Q���)1����]�?��`��GEH�3�A��
S3
�8�A�/.�h�"����~���c?*����oT^]���EEH�C��P<�#��1�gh��E��-����Z������������\Db���� ���y/���c
����XT������9��*�V���:�a������Q@��N��6�`S
�A����-�Z*�w�F�0�������HOAH��!������yv���EX7���G��cp�n�������U@b�0�
����QyvG�V����<��a�<�,*���G���G����n��J��������Z=����>U�~������j��D������0���u��]/������@L����k���6��H��(�E�s���<c��������m����R�>�����E���
��<�����������QK�;~Tg�q�Qhw
��?O���?*-��hA{�����^Z��J�;�"�vdD���5�����H"�� ��������oJ���Fj�w��G��������R-����#���c�>��h�5��4f�{�xDF8@�Dq�J��#!1@66F�@����G�hW���F�d1A�!9���	�_*�����h�
���%����+r\���>���v��PG<�@�}��{���Nn��y>\;q��X����{��ULp�V 	��+G�h}���I���\�����#��!q��LvfS4<���_���?
�~��I����g�9��+���
>��a���Y���D2��JA?S��K%�fe�m�e#?_Z���Z+\	L���v�����T��$��O�d%y��	����$a�,�r����}=([$X�?6b��)���E64���c�uQEQEQEQEr�+����_Z�h�*�4���"�VH���F�F:�%�����z��<K}�i����h�"�yk������<i��?��LV��Mm=��{K�f�-���)�b��?�m�������x����,���(���W;������o����~]#XW����,M���}�S���b�Ku{d|c��G�Z}��X,'�#Fe��B2_��J��3<E��?�:���C�Z]&������A����N������!����h��t�#D���y,a^�E2(�R(�$h�UT`(+����9�[�����p�K�4����5o������=.�(��(��(����������������|Q��������Cz������83�eQ_T|�QEQEQEQE���������6�-w�@�E���q��69��QEY�QEQEQES�����8����"�2A���=�����T��-"QK
��H�������^��^�EVB
(��
(��
(��
��m�������*�k����Z�&�����TiO�9�(��s�(��(��(��(������J������,��u����)Z�s��E��.��,���#���QEz��EPEPEPEP�k��2j��/��j�_���d����_��
���>J
(�� (��(��(��+��W������tU����]���;�S�9�(��9B�(��(��(��(�
�0_Z���`g)��4W�x=�|Qk�wd0<�O5���"������Y�a��
(���
(��
(��
(��
J(=(�5��/��������`���������/C�l��(�((��(��(��+������]a��Z�r���G�zk�����/�x���5��������s�(��(��(��(����G��z��zI��������������	����?�/S���8�Q^��Q@Q@Q@%-%p��������:�V�?�]�V�uV�[�p��)�QEQEQE�|>a���r�8���&�o���U���w!����_7��(��������*;*(��=@��(��(��(��(������/��������^�{�����^\�t}+�����u
(�����(��(��(��u��h�?�_�M�o���6���_�_�M�o���6���������F

(��((��(��(��(��(��(��(��(��(��(��(��+�4��9�[�����p�K�4����5o������=.�(��(��(����������������|Q��������Cz������83�eQ_T|�QEQEQEQE���������6�-w�@�E���q��69��QEY�QEQEQES�>b������i�	�@89>���=�3�w�vq�8��@B(c���8��������=��QE���(��(��(�������ro�
���m�������*���S��f�(�\�
(��
(��
(��
(��;?���R����7��5�]|����JV��\���f�����?�������Q^��QEQEQEQE�Z������}�����W���5?����C5B����������(�
(��
(��
(��
�|U�.��?�]s�*��_�����T�Nz�(��P��(��(��(��(o����(�����>�Nk�Ey���Z�m�OF��zh��������Vz�_����(��:B�(��(��(����J��w�F���k>�5��/������i���[ ��*�
(��
(��
(��
�?������X���c�����u��Aj����#���h�h��(����
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v���w�6nO�y�&�o���������n;��|������JGN������� ��(��(��(��(�o��������k�����z���x\���Myr����G�7��(���6
(��
(��
(��
Q���u����7����������~�7����������
���=|((������(��(��(��(��(��(��(��(��(��(��(�4����5o������.��N������[��G
��(��
(��
(��
�o�?�4[�����o^�^M�G�F���_�
�����>L��?�q�QE}Q��EPEPEPEPv�� [����B�������}�B�������QEfAEPEPEPNA�Q�d����J�,7}����5���6��I�'������'��6�1N�������=��QE���(��(��(�������ro�
���m�������*���S��f�(�\�
(��
(��
(��
(��;?���R����7��5�]|����JV��\���f�����?�������Q^��QEQEQEQE�Z������}�����W���5?����C5B����������(�
(��
(��
(��
�|U�.��?�]s�*��_�����T�Nz�(��P��(��(��(��(o����1\<�)��4W��D!�E���q^{�8�N��������?Vz�_����(��:B�(��(��(����J��w�F���k>�5��/������i���[ ��*�
(��
(��
(��
�?������X���c�����u��Aj����#���h�h��(����
(��
(��
(��
(��2�Q�"���^��&�"����)����'��k�+�������3/�>�EW�yAEPEPEPIKI@>��!k��������O�B�_����U���#�(��d�Q@Q@Q@v���w���A�<w����~�~��Pu��������V��H���Q_��QEQEQEQE
��x\���Myr����Q��������	�._�>����f���Q]��EPEPEPJ:�R�����/�&�7�p?�WU\��/�&�7�p?�WU_aC�Q�G��Q[QEQEQEQEQEQEQEQEQEQEQFh�4����5o������.��N������[��G
��(��
(��
(��
�o�?�4[�����o^�^M�G�F���_�
�����>L��?�q�QE}Q��EPEPEPEPv�� [����B�������}�B�������QEfAEPEPEPJ�������)�pA��Hks�# ��>�PG��D;�Id��qK_�U�/V{�p��+!Q@Q@Q@q�6���O����v5�x��?�?���U���*4���Q^��QEQEQEQEv	���i���o�k�����?���?������}_E�����C�
(��Cp��(��(��(��(����5?����C5B����2j��/��j�}�?�z%?��QZQEQEQE����]���*�|U�.��?�U�����Q[�EPEPEPEP�����s�/�i�zuy���o�-p��q��<�����������g����(����(��(��(��(=(���8��`������C]��������v�����
(�����(��(��(�c�����u��Aj���?������X������??�����[��+�����(��(��(��(+��)����'��k�+��Q�"���^��&�"��(��O2���QE{G�QEQEQE�������������Z���-u�][��Z�lr=��(�HQEQEQEWk���m������5�Wm����~�N2�wn�W��G��+|���t������(��P(��(��(��(�����<.?����&��~��W�������q���/�J�p{3z]B�(��`��(��(��(�i)GZ�O����}���
���W����}���
������(�#�����(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��8���0��^	�P��e��D��GS+�Gp9?�q���|y�o
��{���e���.M���*���������~���=��m�S��W{�����N�,���r����@x_]������e����9�H���W���j��-����_�%�|��Lo��s�dr?B*�������f�7��^t9�����,����z�c������Fk����`|R���ZDG�������EA��M���f�����'�����/�����?�T���z^h�y��!��7�	��?����EA��M������������������B>$��Po�E�5��|%���r���O>UM�����q�S]�:��W�[��2�O�'!Et���K�C���Z��t���/���k������G��?����j�_��������/���k����
?���>����G�����0����_#���?�Q�%�������?�G�������������������{����r����G��������������_�G�
�����Z�g��f������Q]�(����?�V��]���K�C���Z��tkQ���������G��������6���!�����a������~��9���9���:���_��fs��I���<���o�g=S��5��o�gG�3���C��������������2?�kw_��SEz��3���C���������T����[�����C�_��a��[�����+�����O�����������k�����������������G��^��������-�����s�?�s_���tl��W��d����<������PXg�W���9���9���:F���6������'��kg���SY=���C�=}.���>��g�qn�>���;
������,,������(4Z��:Z���?���(�JO�*�%���'RRR����#��m��!����?���(�JO�*��S�/���������?���������!����?���(�JO�*��S�/����������W�?�����s��h���O�����R������������)?��?��O�G�� ����*o�S�/�����������?���(�JO�*������>�>��������ro�
��S�/�������������j�y��+ ���(������)�I�~?�Tp�N�G�Q^��������-�]��z���������w�]n���#oa#���I��{�?�v_���.��g�c��e��h�������~?��G�Q^��������-�]��Z�����������[���=��6��'��X����Z?��?������������.�u���{	mEzO�3���C�����t�=k�;/�G�G�]n��� �2>��J������,��������E�#��<z ���
-N20x.GJ���#�O���4_�^���S��s����y�5����|I�����&��h��#�O���4_�]���������'�����/�����?�T���z^h�y��!��7�	��?����EA��M���������'�����/�����?�T���z^h�y��!��7�	��?����EA��M����������������-����'�������-ybrO�����G����������������R��?����Urn���������/���k����
?���>����U��G��?��������9�+����_�G�
�������}�+_�.��j=���c����~?�s�WG�
?���>����G�(����?�V��]��{?�����W���G9Et���K�C���Z��t���/���k�����G��?�?�j�_���+��W������z'�(����?�V��]W������O�M���0g��Q��'��?�R�*�k����"��[��T����[����������-�����h+�?���&�u����4W��9���9���:?���O�������4?����5�����)��[��T����[����������-����������0����~?�yM����z��k�������gMS��5��o�gG�������Mn���#�h�V��t�?�s_���t�9���9���:?�h+�?�?�kw_��c��S�z����7��p�����)�pk�:Vg�3���7���wE"���7�)�?���O�����R������<4�\Dj�vI[__+���5!ME�CEM�
���P�����U���_�E�)I��W���������F�S�tCEM�
���P�����U���_�E�)I��Q����?������S����?��'�G�)���C��R�T�������A�9�D4T����_�E�)I��Q�
���P�����U��'�����}N}�
�M�
���P�����U���_�E�)I��Q����?�����e����}�]�g��O�\�����
�Hw30r��g�3���C�����u�G*������F�$�6��'��X����Z?��?���������������~?�?a#���I��{�?�v_���.��g�c��e��h�������~?��G�Q^��������-�]��Z�����������[���=��6��'��X����Z?��?������������.�u���{	m^��#�=5��������������-�]kh��e���t_��e�d�#n#��c�]8\
J5T����)FWg�������'�����/�����?�T���{I�y�5����|I�����&��i?����EA��M���������'�����/�����?�T���z^h�y��!��7�	��?����EA��M���������'�����/�����?�T���v�(?�I����'��k�+����K�Ym�>'3�2u:<C ���O�Q�%��������p8�xh8�={v/:�N-hs�WG�
?���>����G�(����?�V��]z��{?���/������9�+����_�G�
�������}�+_�.��j=���a��W����(�����}�+_�.��Q�%�������?�������m^���#���?�Q�%�������?�G�������������������{���s�WG�
?���>����G�(����?�V��]��{?����6�u�|�$���-u�][��Z��?g}ZiI<j���c��9?��7��T����[�����4?���b������<���o�g=S��5��o�gG�3���C��������f��������[�����+����5O��������z��k�����������������G��^��������-�����s�?�s_���tl��W��d����<���o�gMS��5��o�gG�3���C��������f�������&�u����5��Q�<I&�6y������+9������5O������Ke�?kZu��,�n�\yf=�M�$9|u���O���i������e�i��M~?�]����?���(�JO�*��S�/����������_����������*o�S�/�����������?���(�JO�*������>�>������������)?��?�O�����R�����_��4����!����?���(�JO�*��S�/����������W�?�����s��h���O�����R������������)?��?��O�G�� �������x\���Myj����\��.�6G���`T�����f��X�;/�G�]x|��$�%����0����z+��������������X����Z?�������~?�i�$y��?��Z����������������-�]�u������H�j+��������������X����Z?��?��w_������W���=k�;/�G�G�3���C�����te����a#��GZ���g�c��e��h�����{�?�v_���.�������A�$zW�#��F���Cj��^W�|7�����c�|J0�B�c�i��9�I=���!��7�	������
/�:���=/4f�����?�T�����|I�����&��j�z^h�y��!��7�	��?����EA��M���������'�����/�����?�T���z]��]3Z�t����u��+0�6�����q���mPEPEPEPEPEPEPEP/��/c�]����
��"}����pk���y��kHO
����/��c�kkV[���F'�����(��co�i��6Q����X��tUQ�?!V(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
*���i�41����i�,Q��{�E�O�\��+���!xO����Z���.{���_����IX6���n���������06dds���f���e���<@�Q�i������WP3�{W�����>�U��[�����ty�r�B���s���1���
#Q�W�l�1�@�.xe�W�|	�e��eocv.������.���q�W��EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEg��i�����o�����n3�4n����(����������h��%-$�0UP;�k��~,x'V��M�����V�edY�Y��4��#����J���xnkX���m+�tX��S�1���<_���Z>��x�}o\���K����FZF����?�����������R�Z�I�c��!��<��� 2�����*8&K�x���G"�S���(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(������N�����+khT��J�U�I�@�QZ�A{i���4 x�C��y{T�QX�������
Z�����4�f�(��UcF���i��C�-�������C�h�������$q�,y��@�l��|\<m�}g���Ww�[p���V �p3��r:o��m��F����������C=�-+�:p�����_no��x��:��Y�Ew���6,�?�(OU'}h����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�����P������n����;��l}$���{�J�+.���U��/�KK�&��*=����$`�zf����>����i�!�����L�Ebo��5��G?h�P���"�����2�w'��Pe��\k������q��f�dyy[��S�1��X�|Y��b5}W���2�4�3�e�\����i_-t���_J�4�m�c�G4g-�r+���/�/�:��������i�����a�n<����C�&m7��5MG�wq�-��]��gH[q��q�������<���|t�6�8�~��F�%��G�G
�����ko�;��Mw���/tK���{�Q	�20�1��Y��k?j��?����=����)�M\���u|:�/n-�_��{{�h�4hH ��=�p5R����s����5+�2_�����<����|���]f��_��4�_�OXG�F�d���,|��Msg�;�*�D�V�=T8k2�$+/U�
��s�ox�D���gm�{+IZD�v*�I�S�4������K�&�o���,e���-�g�=A�]'��3�;mfMo��!7��	�;KH��X�N�6@��k�>����%�����wi��Mn��t��h�>j���;�;l��M�?�ib��y@���8�h�T�T�����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(����~�����e�w6�
������q���E��I�<]��7S�Q��$�!�mD�<�}���_]�X-�ieeR�*�����t������t�)���������r�0'��s�;T����<s��a����)q	R�F2�#�=+��4f�����	%h�4-F{De<�nK!�$���Y\���os<-n���<R4d��9�+������=Mt�����2���J��YA�z�<->������o����s���H|�`�opq�g��K}�o[XG��V���v��6We�z�y4��:���������K��l%���3L1����
?_�>&x'OOk��]���o��k}�DlpJI�����*���c�����0���v�}����"h�+��5�������sn�h5[����?�#��>���C-����k�CC��Yn-m�Z�
x��zw�u���<>tk�~��i��P�y���1������"���#N����Z^X1�g
C�4,Ug���ZI����[��q��"u]��7.=pN���@��_����u�u9�8�=I�C���+H�����{�����w����5Mf1
���Y�$�1����
�vh�ius|v�{�<�����>��*����is�>����K'�x Tf��(�m����M�+y���"E�
��v����+���(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���)��/�����6�%����y�e~oN�U���>
����y�;(��d.HV�,�~��6n�I�Y���A��	��F�[��c�Pe�q��w5N/M'����iW	vKt��X����8=z��>2cK>����������+{���h|G�u/k��-oe�I�Q���$Y��s@���i�%���/��m�[�B.}2k
-w��?���&�o���4��;7�9��q�t��o�v���n���vF(m�c�E���]�<���+WB���������j^�Pa��X,)3g����r��O��[��~��6�j2����j*�8v�����@����?x�\��.��Cl����G=�";��8�8��/�j����
"��-V�z���M8��|h�g�W�u7H�#���'�J�|5���g�/�<�x��gS�"hl�S�8����
�|�OZ|A������{M�-���2�������
6�������|K�������l���mCL�CyRH<�e���Z���s����g���^�����M���Tw��H�,9��(��g���R�����t���d�����%�2�X���R_�{�D�����u��Bt�o"Rc�P��p�Q���[��/���c]{I��s��+�&�Y����q��[�mo�E
U�
�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE����j�Z6�>��\%���o�Y��4���k���z?��Au��"�����
�q�����(��f����'������M'�O
��s��k
��|B��.����h�4�xsV��O�5���$a�Vn:���Q�����m3���mMF�I��2p]z��zw�r�>hP�����u����������n�>H$rA�cx�U�Q������������?�����G�@O��oT�����i�r�gm����I�M���p��g�F��.���4���#L�dl��6�}�kg����=s�����^��W��*X[[HI!�r0$�����K���u�X���Ko����4-��=��z��&x'H�>�	oemiq�������#0<c#tc���P��>�_��c���_��l6������'�+_��:��->�O��H�'[s���
�H;3��<�5�G�M%<�{a&�������������	�jQ��oX|����3Q����?:�|O��������U��-�%���Q���q���t�];M����c���b�.HEQ�9��Y�?�����������B+Sige`	����c�7'���t�	��?�5mn�L�Z�F�L�(0^�����-<�3j��7c�zf�EQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQFh����Y��/�w�SCw�jw`�}>�/2i�,p�&�����?5��<1�hB �#x��p9�6�#��@%e�~%��7c��wR�����<��z��V�T���x��
�/��S�������Ji����r�;:��~^}��tOi%�7��-��Z���?���d���}5���]��Ki5��m�O'Eq���o�O����[KOo����m��!����{�~�����f�c��Gc�x�4���f	�p�q�����_P����MT������+���X��*�?����^���/��O��d����z�7����PzJ�*�����g���������/�-���$ #�m�p2+{������7�~�������!]U�����][ 9�j��+(|/��������������'R8b;���T�����F��o�m-fOC<<{��5��xY�����|u���]i�������x/�r[�]�]��c�^YG-������wE�a�����M�|t�N��H,�{x�(�!�$��	�	���m�b�i�-����	fT�2I#m�����85����G�t��Sxby$���Jof��y>�q�����U������r4v���Vr�(�����(�-PEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPE�f���P2I8������:d�R���Z���<�u�bu��+����6�O��~b+��a�#8#��	sX����������4�����v�P���z�W��-B������s-���z��F ~\����^��?�5���e������������|��$���6�i�!����=�\[��%��	6�}��s����&]Z�Z�����������[
r�d��]95/~�:��t�]j�$�A���x2��8��K����K�tz����*����L���6�d���z���M�O�	��S�H�yd���*�6�>�����G�W�5�6�`�-����Fg��wfPL�����}k��?���}g�:�
�RMf��9�d]���M�W�~*�~���o�� ��U��-5���P�!���P_
Y]x��G��+{#\M�Kqco!����B1�U����ip�5�/_�f�<���'�������O��L�w���i�ndi�c�o�+GA�����,��%�����)�1`Y�X�OZ��|M�
K�����WJ�U���6��<�
�3z��OJ����A����=?J�t��w�e!�Ye���zpP�����K�
i�^2��<�p�,���
��dn�����^4�iuV*r����Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q�(����Q����6�>���H3%��ngx�����@U��ge,�]��?�
� S+uA<�aY�.��l�1wq�K8o�D����_�q�2z��|_�5]c��?�t=0���S����Lw���i��9����x_E���sw�?j��f(��Ga��G9��@}����R]��S���B69�9�
��_�-.n�SS��b��M,�/��Es���o>x�GId:���v$�K�A�^A5��"k	�������G �$>}N��x�����������f�1<r�a�@�x�z3��+������#����R�F��]�E�i��|�(-�������3���~ ���@:Dw��A�E������Z���I��6n�+��$Ry[+������E�|X�1y��Wk�crm��E�^X_*X�1���$��-��i��
���e���
?�t�{���t/��.�k$z6���Cr
�LF�����V������Z�������l�+4��^��������|i��
z��]�-�l	V��<�3��H�Q�awujYn���'O��O$~?�v�.�c���K��Y��#�0s���{���,#�4��	�1���8>�������#���]7S�O;X�G�&8����k�t��i^��c�%����\�:���w��S�?�&���l4;���5��|�$d��+��wB���3}%�����\HicO������W@-QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE����Ziz|����v��!yf���(�MqZ_�[MwR�-���������������';}�@�������ZZ^�=����94$�C�YW3��<yiP�=�y���>~Q��c�W����������j������f����.����5�|S����:����n�9�C<�W��a���q@���^��R�.<�EU,�9����O�r�?��5{KkH�4|�-.5+o.)Oa�<g��>&\��	���Z���>��/���������g��&�E�U�=��)�9����?����x���N��h��������$���iC�����	�2k+����l6>0����\k��T����[�X����E'�%�����<e"���g}#���?\�zW�t��]�F�H����}�]�<P7�_
����V~4�5��.��������"wR9��|g����2x��E��uk��i��H]�
�V������1����k�k6��y"T��U+���F����V�����}sP�&�hw[5�
���p��;���Y�'��.��w�����
.��3���}���0G ��W1��/�ot;��k��n��#X��`Z����]k�^&�����K�����1���vf��J�����#�[mwM���N�[��m>�����
�v�c�X���i��v(#��j�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�//mt�W�������� E_�<
��W�t�c��]��9���3,��P��{���a�\�F�[�[xC����U��2(��;������e���K���3B������N+7�W����{K�X&��8?h����rq�Olw��Cx��Z.���~'����1�2E\�1#�I�O��y���~3����H
�j6�W,|� g�8�
��S��������jz�C3[i��;B?���u��'����j���Wa�cGA��x%�v�C���#�(��W�"��Y5������������
h_���L
�kz;My"1���q����bx���������]��F�Jq\N�������85����I.|W���z����FE�������'�RZw��h<o�f�c{��m?�(��=��[A��~7h�!�������K=FE ������zP7�t��>�m�
7K��������{���L���u?|ms��]J�V����i7F��}�O�T��'�}KV��/��O�&��p�`�X��r~����<��3Ik-3�����qsp���C����.����t�]�k��5�,|��+���_�>�����_,{�V��vIm	]��!��v�"���)����.u���k]KP{�-�{~���<��x8x'���Is���e{���c���X�n��|7�|���>�K��`�G����>����j��v�98M;Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@f�5���������g!s�s��2X�k�\w��)�[�����}��^HC���f�n?N���x�L��a�}�����l���"M���wn~`�	��P�f��x�B�tp�m]��q��1�&���^������~]��#>+���2[���-��`��>���lx���&�����)��kbKh��s��!�!����u>����V���z���}�����
������x��$��o�l<c���if�,�A����$s��5r/��&��E�i�9��Zm��p-m��oD����{�7���#�g�����%���G��KD��y	�����E����^)�X�//���t���"1�[;$(G#����x;P�����\��a}�z�?����{�|w���.�r|9�Z��w+Uu�A�����Rz��5_�W�x��^�7������7�y47����Mv#���|7��R���g��������������|Q��c����[JO��j��zw����
����SY�����F"ig;#�3������4��5��m����\y��3@?�4�����[���C�\�t&����9<T#�>���]����N���-���$lz��}���Q@�����3I�����k�L�Ws��n$=Y����kK'��-a��$2J�����6:�sV(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��3ESh�tGuVs��X�z���U����>��]jW�����-����x��g4�Q��������R��,�'1K���	1���+��#?��*C{c�-n�J�F�����/Iw.���@O&�=�5�_�\�WR����f��;�w�)��|��������fx"�T��q�|?��[�J�m���wp���nw�?�T����+��Q�t=;����3;Au5��#�1?8�����;�N�]��#
�����Q�X�]IH�2�7Q��r+��M��g���y�x���m6+U�����a^s�uI��x��>3���w�W�4�b�;i|�}�J�>�?X~���4��6�7�l��G5���L�����#k8���I�P��z���������4���I�cS�2�2�`�~U������=�g�������|���+��xZ�/����|i�����[u���������`�*ko����j4�;��m��BZ"�x���C�������=�+T2
PV�h�� ��v�h^x����-c�z����X9>\m��1�q�u�&�c��~�F�F6p!@�9��NIbz��Wm������ ���H~B�8?xP��������O�	�����cd8G�b���y��?x�X���vS�;*#m������L�� {;il��%�Oo�/��
`t<c��QQB�
�0R�@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q�(����e���{��Ckm����U\u��o�jQY��H�g�q-����z#(��;�3^�S�����4}GL����k���������
s�H<�\���5����x�/��k���m=P6Q�F��s��������O�������{��'�rO��/|H�|Q�-���k+�:]\��P��������s�(�����x��������/F�v�c�r��z�8�7U���X�W���
&Qq-����JN7m�������7E|A��O���B�0Yg���~�	���q\�����I���;����K�K-M`��H�����p3�Px��F����/�h��)>��Z�����9
 ��J�����k�,�������:��c�L�0�e`�Z`|�($���@'�R)�;��MQC�V�B�)nW�=;g���0� ������;���-|E�j��Z��ykwYc���z�h~��^��<[���b{BL��-���#qE����S��&���)����(4��o����
�<2s���j_$�_�)��
�5���YIku<��Tv=�}k�o4=+P����M���/�r��'��WQ*(UQ�����
.e�L�#��.<=�]([�K,7�2����/jwz���\�Z�����.�F:"(�W�u�Pe���Q{���^:��F<������j��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��+�~+�@�7����@��� �,Y]X�#!��y���Bh����2�A�A�r�v��|J���\����x$��<.W��k�����3K+���4��M��!���@��+�<<�)��ow��xwJ�L��Y�������#7$���>x�\������cp�����5��M��������/��T�������Dq����s���v�s�^������4�V��_���v��d����r�� t_�g�1�]
������.�Cs�/�<%y2Dt}NI!�.`��F{�b�����>$x���*�n�G�n����Oq���,��2x����x��/��,�h1j��.�f�
�����8�����?V�4O������Y��O��I�W�?�G�������oGi
�X_Z�%�����*T�H��.��������-�>��,�=�76�}��l\%'$���+���t���K�V\���G����>�*��+���V�>!����[������y
�$rz��@�3]~��
.�������Zu�+����n��~TS��(�O��[O�y�qj^
���n\~ Ve�3��h?�w0�z�q"���m�� �989��8�(�8�"(
���N�y����T��
����6�q���gix$�������k{�i��C��Z���P�
�����\��QQB�
�0��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���c��j~w�m����!�S��8���[�����6�CYlD�����Mpe��|�8���{������o���5
j�+X���6}������G���*��-����$*l���s�
�-~4�R��d����yq^�X����7���Y���G����8Kk��\���R�<�����7>�����to2��n5+����0�x���]���f�h~��|�V��n��/�����>+x+N�����G��y�X_j�����Gn>UL}�'�:^�e�i���U�v��.���aPu����}s���;�h�[�9g��e�����q��<����_��?��<A.��[[kKOj�@d��@b���Ng��������/�%�Rn��K�09�d����sE�F��/��=��6�n���P���2m�s�b�9/����6�������kw`��f��%F��p9�RM2�S���s�xK�����<\j����o6\�7�rj_�fm;E��1tv���o�cc�q��������'���?��@�=��+�s[���W�c�
�|3���h�u-i�i�A��g��X�:�G��}�8����?��������kv�j�����L~$������R�[�5+�W_���S�����hM��`t_s�]�i�O�����5�hcK�2*������C������G�Y�r"i,��x�!a������{�^5�u��4��3L�������E��@r{�O�k����;�6>���<����y���o�[��I�`+v;+X�x��I�������=MOE��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�����3k7z��|=��V�L����S$2�gj2W�d�������������YC<7:-��g[�,FC��}k#���e�N�5���g���f�m�`�y�z6?<�$#���Fx������"�xN�F�������F���0��~�-��BU�~R2s��WG�K�h���7 rb��Z�)�97�=k�<9����i���-#���/����D��	�S�?��#B���6]"�����n[�@_v����2I����_�6~�������1��kp��Dm��`�d����3��?h]sO����Gc�;�M��{�q��a�+%���,eI���"��0lE'���~��lj�����]6�`��N�!�R�����$��&�=oF����yq����{D������m�JQWz�s�����^������������y+�����?��{����|;����OB�}�����+�6s�pH��f�$��e��q�P��n�!5���!Z�_
jp�����~�:j+��r�\~y��O��o�4��3Oy�����s��|�����4�-N��J�������v����@w��I�>���R�!S���C�����6�~U��^��s���^*�m���s����N�-|�� ;�N=��1ET���/,V������@�@���T����V�ogv��HW�5QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEW��t���{}r��hW����Ca�����+3�zJk��t�Wr�[<<�� ~��1����^�����6�$�Y�"?,�FL�;0���>��l�<Ki��$�f�j���<:����G�v��5�7��������^Z�BQ�[(	�x$t�Tto������N�5����K(�D*Xa��pz�#�9?x����������K�.�ag�X��f@8#���|g������ZOk�C������ck���p����W��s.�Q]s�0��+���_
x�WMO[�R��T#H$d�t
����]UOO��t�k-:�+{{T���5��r�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(
x���M'���_i����i�����T���U@y?�{my'��b��.x+��F�	�m:����y����1~'x��{�[�xcZ�v[�����a�a�a���<k���F�q�n����Z����,2(o���c<�����^Z�{c=��ts���=�?������n���>
�M����WS��$���"'*���ns��(����5������k�zN��Mr�E��a�Wz�����Le�_� �q�� ���M��!Z���g���84_x���=o�~���[����u��wWg��i~����-e6�7N�������`���k����yK�i�$�����c4X$s��+����������F�[k�mci��v�>�����gek������V�F0�B�~�T��E�6I2��+�=+����7D�WR�<=e����!%�r~_���(�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@x�����o]k�v�6�
�2�Y���I$W���]�[T���i���6ZQ�&wS�X���x�\P�x�����2�����	�pCg�N��W���Nw���]'��j��C�o�]A-����#8��������^�����r(t`C+�QXZ'�<?��}+N��������\�xH������y�_x/��%������n����o ���99<W����T*�����������|A`.���Ev��=@e �����/������t6��O5�����$�'��k�@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@d���O�����-~���/���^�*A�Z�P��K�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���
logical_decoding_work_mem changed.JPGimage/jpeg; name="logical_decoding_work_mem changed.JPG"Download
����JFIF``���ExifMM*;J�iX����>�Cherian, Ajin������94��94���2025:05:27 19:56:152025:05:27 19:56:15Cherian, Ajin�� http://ns.adobe.com/xap/1.0/<?xpacket begin='���' id='W5M0MpCehiHzreSzNTczkc9d'?>
<x:xmpmeta xmlns:x="adobe:ns:meta/"><rdf:RDF xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:dc="http://purl.org/dc/elements/1.1/"/><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:xmp="http://ns.adobe.com/xap/1.0/"><xmp:CreateDate>2025-05-27T19:56:15.941</xmp:CreateDate></rdf:Description><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:dc="http://purl.org/dc/elements/1.1/"><dc:creator><rdf:Seq xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"><rdf:li>Cherian, Ajin</rdf:li></rdf:Seq>
			</dc:creator></rdf:Description></rdf:RDF></x:xmpmeta>
                                                                                                    
                                                                                                    
                                                      <?xpacket end='w'?>��C

		
'!%"."%()+,+ /3/*2'*+*��C
	
***************************************************���B"��	
���}!1AQa"q2���#B��R��$3br�	
%&'()*456789:CDEFGHIJSTUVWXYZcdefghijstuvwxyz���������������������������������������������������������������������������	
���w!1AQaq"2�B����	#3R�br�
$4�%�&'()*56789:CDEFGHIJSTUVWXYZcdefghijstuvwxyz��������������������������������������������������������������������������?�F�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(������/����������,r1
� `���<�~#|E���u-/��6sc�o� �y�
z���
��������O�kH ��d d�p8���k����(����j�h�-��vO�iL��B���+��?|O�}�:�����zL�~�n4���Ns�{u�\�QEQEQEQEQEQEQEQEQEQE! I��W�j.�
v]��������m�y�Q�g� �����Dj�|ksj��,.wH�����S��</�xn�X��Xb�R|����{�Z�QEQEQEQEQEQEQEq���S�>��	n�,�[,�pc�#��X��<PaEPEPEPEPEPE(�*�����C�OD��Xt���.��!�$G�3^��������N-������|��8�}�@�Tw2�me��B���7�w�n�{�����Ch�]B������������e0Z�*��F`|����W~<�n��l��xn<���0#h9��@�Q@Q@Q@W�j�����w�g������*�{1���(���(��(��(��F;T�A@Ex���_�>&O�x>��;YLR2���=���jX|_���Ao�x����R\�"�o������pN;���@4PE�xW��+���[Y�}&&F��9��v�:z(��
(=+�����]O���������O�<;""L��[<�=����(��(��(��
;�Z����O�����6��M#"M�P���1�]�R����/��.]j:��?��	���%�-��GQ���{f�=F���|c�CG��Z���nmW����w��U��_�6�*����g��t�Ds
�R=���Q@Q@�x#�z��> x�G�>��]&.����c�u��NxQ�W@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@��o�%z��{���|��K�q����?���������
����
m|G�M7T��Z���W-omp���`{P�i4����t��]Z���G�����E�}3]W��}�����D|c
��������U�����3�P95�����T�q��$��.���}�c��}�5����}������{-6�o�}�"�D�_n���Z��=W����������t?5�6�Q�N�����^�m�(5{H|�-�O��HRpG�#�\��>-_��gu�������~��r��s����</{���7Z6�q
���r����k`9��7��?����[h0i�]����w1��0~�*���O=�o��I��wx����sl���h����Wn�7v�_��$O�;�p�{)c�p��WK�G�Io�������Y��G���H�J:?������ko�?������������������3����	���_�I�/����1���c��]~9�W�}���=vy�����	5}����M[�GK�4�`.V�
4@�g8������]~�W���4����<%�O���������
�^�^�+�>|���q���|"��F�7;qw����o����o��>;������NZ�S���1)b*���={V��|A�o|/��W���u��X�h����|�/=i���j�����M��
�����$�����@�u��2�b�<4t�Oh����Z[�9|`�Rs�����|X��w>�����0�x�_�%����8<�b�C����_LDUl��`�1��Hi/1Q���8�O���_���E����WJ�Un!����r���f�^�P[�=CNj��x�����>2~ky��_�9*/�����:n�2E���3���?@��m��S_�}�@����(d+�[
��x�_O���yg:���ksm�w
9<}3�� ��'�����=���n�e��mv�mg8L�������G���~���G6�pWw�4{%!�Ld��h����MW�Z6�w��
��T�h���ac���E�>*x��h'K�,$�m.���'U.p@$v�C��P���S����^
0����e���G��� �K]kJ���DW#8�r#���{��F��|1�;�u�����*s�}O��X~~=��PO�x��3��s�n�_�C�n��O������{xE����t�������v�k���K������h`G��O�@���8������
���+�A<�Ll|g��7O�~,x��um-���a���.���8=
��_���|S7�<qg
��~lA���:�\s���EfX�?_O�k?h���� -N6�c�=1I��<���t�x�]�/�J�"6F1��Q��~��@�\7�fm���-����e���p����w5��4�u/��������� �V��BO�@>���}4�������]���.���|+����h�B����E��)��5��B��A�����[�[#�)U�I�rRk�4�|J��%��*4�HV�B�.�}���|g��S�NzE���W5������a���cg�c�?��������-�^�����+H!��f���b9��c�88#��k���5?������������c�����?��o�,/���?��:}��b��o�(�	�����=�+@..>-kV����ZM�2�M�w���n$`1�*���W�>.�a��=���&�x��[���<��$���{��^A�R$+���Q�:2S�m���{��]���2����}0����i���:���k����w�xU����\�A�DdM�����Z�h`?�\@�������������x'��@������>����CM�b�6\_���&�!�G8�����x�K��~��on�wC6w��	)�=0p@<s�������S���|����om��q�6��a��������{dX��z�.<O�
�6����Yv��������+������[���:����g$����Z?�e�h��?�|*�
[Y��M(���8<d�����w��#�o�&�E'V[eG��_�@g�|e��.��C�k�s
����z���#��\6��/����iV0j$*����f�
F��|�����E�<����"L�u����~o��K���O�{�,�	��2�C������v ��_�Z]�����FT�&F:���j���j�+
]�o�^�0�1l�r�y9<
��/��|]�a3���%�����[�-OD�
�YZ�RyW���s��s�A���k�p�P���~ ���=.F"�Ku)$Cd����j�
�A�����?u�����}���n��������'��N�������B����D��������-�r:c$g��h����x'N���������v������q�Rk��|h����n4i�W{i����A�	�5S��?������X���v����dtP'���px���v��������l�c��=�o�����_����w�U���2?���c��c�W�*���7�&?�g���_�����~��5���T�Eo���@%c���Ys@
����~�Mo\�I����E�6��1�p9�#����3^��jpk:=��fs�+4y���o���N����l���������@����h��9���-KN����e��t���� S�����'}k�O�5�~��
1�����1�>���]?�����g��'Z���5�~k�-�*��s���3��v'���~C��
�����c�+
�V���k~G�
����}�O7wM�Ns�W�~�o�+y�����}�����s����<|������I[�w��n��2e���s�W�zP�=���M~ O��F��n�����������9�������z�+���:���7�5���i��
xR��]�-lf������_-zt�?�{�������#^)��:&��F}_J���/�+�b����=�Mw��<-���&��4l�$��\����"^��_��"�m{�	���'����E����#�t��/Q�����h_�=�.�\��}e��o����Ul8���?�	��SY���{o��y��wL��u�x�<�p?���K��>1��������-6�o[�b�"���|c�=:���%�cR��,�<W�IoqE"}���#}��];Y�Z��:�����m������>�<����MI~���H�"�a���l6:�����U��_�i"��L�I�[kn�rFG,�k@�>"��I�]����m��������<��Mnx3�:��&�?�vpA��Fkk�n#�Q���	�� ��������
�G<P}�P�����9��rq�G�H�)�j:���=[�Sk�>Qo>h�4j� ���B�����\~�_�zy�������������5��m��6�Sl�t0
�5��wUMs����H�x�V_*O���*}���lw�����G�.O��W��v���s���=����(�|������=�\�������A��B?]�c��B����t����O��U?�hF��+�.��<3��F�����u��x�X�>3^�[@4]*���.�c
�>c���������u1��^F�H��_����i|
��������Z�[����@H_��?�G�x�����o<+����N@n���	�zrBO5oL����k6_���.��<��>J�8���d.p;sL�'�m`x��]]/Q�P.a�7E.�=p8#��Ef���0�^�ce�J�{+�<��l��}H��\pq�@�M��[}
8Sd�V�C@9�<�/���
3����}G����Mvd�5��{
�:��_�=���S�(|ck��6��i����c0@���<���w����
Y5I
��}n�k�w�n����=(}u�y��a�8,�U�e���8`.y�=�sZ���������h�����.!�2��:�p��U�������<���l.}������8\���I�����>���?�7�C
��M����I+���t�@�:���>���w����E�����V��~�����x����3i)v.��`nP�2no��3���I���)���0�����
��������Pia����E���kO�Z��V�\)L����Z����O��M��q4�Iv"��������p�j|{��V��go���1������|�����o��vm�����7�b�����i���������T�n9�+�WZ����~'�@���5K�[F��48V.���+�J�F��Gx����m��L����k���_Y�sY�G�t��~�om_,��u�y��R���M[��������N�lR2s��?J����[�������V���d&�q��*|c�7>L�px��/�[��6Zn�m)�u�,fa�9����i�x���
�%���,���BO*N�mX���z�=y��o����
���r�S�zU��Y�A-�gF�wn;O9��jx�����X�g���G��)D��Bt9	�@���Z�#����)�K�Go8�$={	'�W�M��R�Me%�m�E�#�'��2�@'=[5K��4K���m�-��b@L��5��x����u�<A��T�.,���}<�=��b�<�e���t�����
A�/�co��l[�_�5���|b���>�2[;G-��Uu<�s��g���C���
��J���.��7#�}*���.]O�u�������*IUo��$����V�W�i�K�40y@?u�� ���h��Q�/lt
���e���<��P0:��R2j�p<��T�����-������J���|J��&��,4�HV�B�.�}��Z^���Yx�|!�
�u	�u����s� c�88#���|3�?�Y�e���$�?��_��k�����'�>���l��|������c��������f�}'�gns�n�q�G� W9�c����h�������d#����F�|F��/O}��=�6n������{j�_���k?<k�����2���P�Hp}�u^7���a�o
�6�5
z�7�����=[����k���G��d��}�y{zm�e�X��x�����#���i�j2d���hq��?J������}^��F�-�_2{8#*�$���zg��A�����]��]gO��)���2����9�zH���)VmO#������
�<�y��R������F���T��|�s�O�@�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���{�xU������dbG���<��*�z�����:��][+�O�yv#<]5W������<W�	w���5v]�0L?ur0=y����k�(�.7�n���I����n|���q�U������!��b�;�U�d�[�U���8�rJ���8�����!�u����nE�������<q�]k����<!���h�6��}Oj���9���%��|��:��][F� ��.K�����_�?u�������!���������1��#��A�L��<W�>(x��|E����L��5��i@9�2s����5�������)�mZC�)'���}�b���xSS�w�����n~��b7��p=
I�c������^������LM&���y���������i�Y�M�2t��r2\O���i�����O��G	Yr���W}�����~"�W�%����b3ur�#���Ec��������5��5�#���k3D��8\d� �������^��B�-�mJ�V��yH���L��zqM��c��~&h�&�/��ef��2l��`�c�� v�c��<�����oi:M���4���i|�v
�H����z	�h��<��������&��c�(�%������7���u�j����];�`(w��(��<���
z��l��q����-M��[�rG�]����|K���x����I��`���/��3[�P�i�W�o�
/G�����n�NRDN��<}Mo�f��~"����e����!���`��:����)�����L����ea���A��@D�<W��b�����n�W-�M.�������<q�V���&��~�����5�n]�z�K*�:(���$�����T�"]O&CJ\���OS���I������]7�����Ar�-��x�}21��Fy�5�4P�x[���{�E��+�'�!]��v��\��'w9&��R�~���<>tu�����7�.�.W�'�^�E��
��xc\�M����;�:���y+�C�����z��\����/����\-�LD��]�Xp}E?�lM��f)1�,�NpTWoX~3�e�?�u-�d�[����E,�rHJ��i~;���t�����c�B'��P%L��(A�>������T��~.��qn�v�v6�)�}02H��5��CD��~�����y,����+�@?Z���g�5��k�KmkI��C��dg�Gq�H��i�[��
+Z]+C������b�H��d�H��z����x"�Y�UXU.n���So��Rx=*}���������o�DP��.[u\q���zW�|.�~��7����/����|�w�r�x<��(�n��R�x���&��p�4,HS�s�q��K���>.�u��X��i�y�XY�� �y� g�x��Y��8�������i,����X?�is���8��t"�����<�f
e��������:�Z��(��{�H<���Y��{r�m��1����������_j�O��x��`�nR\����q�OZ�*(����w���V�#7WP��H�W9���/�t{���V������$���OC��
(�9����KQ����{��j�XK�'Q���9#�^�Ey��|X����uh���o��{y4��c$��g�^�����W���>��
��{}�br�}�$����<�xg�'��V�w���R�%w1�`c��A����6��h��6Q�R+��=��[�~����k�2y��n�p9�5�4P7q��sF����������;V�g��,���f���_"_0m�\��
��_�=�`�k�����6Z������~���>��g=�x5_|e�_ i~�=
��[�n?��F��e�������?�k����[�.�\e���q���j�4-7������t�]Q��)ou<�L��A��������;�����'��&�/��d�K��d���e	c���v���7�#���N-5I<�;��#w��r9]��6���k5�g]�c�mX����Pp[�*o��?��[��x���EKi����
��1�<���B�(4Q@�_�^#��e����������K�y����.pr�����YC�i��V�h�(�vU��P����SK��f����i�J^���������u�s��|@�t�<|t�7J��L-,��H�\�dq�x�W��@�q� w�P�|"�^���b-mm�����y2��v��+�h��<�����&��2�4��S�/&{k���@�^���� �-O��|u��c�s���x����r^���+1'j��y�+���;�w��
x�J�%��Y�2o\3ds�\����k��j������m�i<���r66I��'�k���Z�H�|G���/�6�h���d�(L����Z�t�|O�m�xq��{IF&�o��	���}2G�z��x9<s-�����O�@��v\�g<�n�q��o���i�<A�+��i������r~^��:��4P�4��{������77v�BF�����U��\h�t�X�&�Z9�z�g����$���6��r~]Y��2�u#�>���d�sWc_����Q�b�6����&IA���#�^�Es^;�m��</.�s'�.�%��g��t8�9 �C\V��=6�4d�E��1����!fUFy z�k���(�/�Mk��n�{���������_��3gb��n
��+����,I</���VV�^J�<W��b�����n�7-�M.�������������(�%��&��~�����5�n]�z�K*��	-����z����H�K����233I��j�(��|3�+���M��i���\�x�,�L�{�����-��^�������j.�K;c����N2ps�k���3�?����<mq�=N�O6���
����GB�r��yf4���Y+/���}��������^�Ey�������<M�n�oau�-$������.00H`j��>�jz���|�.���.�d��P8
��:GZ�*(���>3I����?c��O�����
Y��Zqx��+�H�����P���8��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�?����R�Y�</�[����O����6�?��@=+�<������'����J����A	����@�����#������n�l����I���1�b�|1���<�u=b�����$���@�@Y���$�^��	���6��-!��UII'���;�+�l|E�O�z����������Z�yA����H��in�I�/����~0�����<���A$��?�\r3�1@�Es=��>�|��E�RDv����7L�LO��'H���=sO�Z���z\�
%�����T��v����z�r42:���y���2���VW�����y�+��8�f���?��d���.�"U1������
�@�z����mc@��)o��5�W��U��F�r�����(�h�s�^/��O�n5{���H�s�2C�g���}�J�>(x��=^����%��!���1U<����'�@�Es>
��V�mwo��R�Te�~�s{�909����uo
|F��{�����6�������{�}�9�h�+�������}KK�����|Rm
��������?��$���_�E��z���]yA$��s��s��A�����������a@��^k^�5No>���<�m�����W%���In��^����h��^E�x��^9�����	}�a����@���@�����#�����[S�g�/-�S�m%�t ��5c������MQ�e���mId�s����y?�8|m�_��k��~"�5X~�a����pe����u<�W�|>�����wt�8`�VI�
��?�u�W�i���~(��������e1���I4���`v������o���3���4-J_!/�A$������ ����+����x|��~bs?�k8�g�`O�\���>)k�i��~2]Y�:YAjF 61�����=j��<%o�=7H�/�6���)1�B�E��s^yg�/|R�/��������P����1����94��W�]���u��~���_I��($���;��r����^1����YtY����<I#}�};�a@-y��)�W��>��)k8��I��';�����}A���
:-r��1�S\(�jO 7��cX~��MO���>!�K]N����V3�;*�}�G��=/���GP���
 [���&Y1�������5��k��*��X����(tk������r���bn�c�x��Z�����~�wu���mix�B�*��H��
v�W�i0|E���g���$�����J%��k�+�6�`H�j�����"������O���^u��(E<��0#���=J�����9�4[�7�~�$�uS��p�3��<d���MP����S�'���%�y���[�YTr����8�P�Q^c�c��������f������bB	������:�
(�6��6��|y��/o��]R�u��r`�lg�	�1@�EyW�/��5���xF����I�e�91"����'�����.[����&�p��i�E!�6S��WEy��mwX��.���A����H�H�P��y�gL�<s�WQ�������-�1E/�IO�''�s@�Ex�x���
<Q����P�[�5���x��������WQ����<���x��_i������Pw^q��:���?X�W�E��vc�O)�]�1�	1����m~#x�F����~��%6�Gn$�Q���x���c���O�h<g��+�|���U�_��h�o��@�Ep^8��t{�������|Ks��o����)?�?�[��5�;G�Z�����g�h�����Q�rz�PAE�Q^-q�?�|_�|1��G08KQp��Y
��)��l@�XRx�|E�um�g�J��i*�����8�zv��8��=����f����o�mZ_���i�y���v9�Fs�AK���~!�����|gp�������j����3�8�Ph�h��'�u
�O������_�!r��q�����k"O|R���C����Q���lL���v���=V��~x�Oh36�n-uK	���*0��v�;k��������^#���{�e���%�a
�,� ��=���O^x��|�Z���1�&Y��d�FW>��p=G"�R���>�������>^��m��$��6x-���R���������oX����&�t��\ �L��U �d5Y�����n�COZ}�.�u�?cy��)���~�u���4QA�@g�� '�������-kPm��Hz����q�=����^����O��"������W%}��?M��5�g[Ygd&���%��.B�����5����T�w>%�.��i�0Qr\1\���"�=C�P�����������9T~�u���u;]I��u	DV���$s����3S��4�}GM�g��A$R/F���4���|D����,��t�[�jT<1v��O�4G��|G���i:������2\����=��%�`�c�{%x��1���Fc�==UF��{U��1���?|E�t�-�������sL����c9r���x4�~$����"P1w�4jO���S�(�m����9�p�J���FR2
y/�O����|F�_�����m��=�L�7�p�s������i�r����]���ku��*r	�/����%�������D���V���) �~8���j���t�R�F..,�w�]�s��J��x�������4+}����:7C�g����
��g���6?��w���/x7J�-�Ay%�������y2�q�A�@U��ox"��O�
�������9��~�<�'��z����[G<)P:8��FA����!x���u������77)#�2�(�s�~	��t��h��?��+
?��Kh�����?�]��^e�x�Z��3���
�\�������E�>��T��s��@�E`���+�ojZ� =�'���< ���[��%���������^���S�<F�(}�����4��4������iQ�/�
���eU��~Y?Z�<�x�K���������x��pw�����������ty���km2D���<�������(������<q��^ ��t�5I�D-4�Ym���+�+�~'�p�/����E{5y��5�I���hv��(��R�'fXd����V���>�]W���N���q�<�D	�s�����z�p<Gg�|=������(���=K���s��u��i�-�V�����bD�0G�>��~����O��7�����ig������H���v�g�OA]'���@�k�X��%����n����>�0�
��)j����D������In-���FK�o�Z�|i�����x��M�4�d�5�U�'��z�#��n��
GO���}�\D����V�5��:�����2/�9/n�&G�i��H�X��' t3��@�����?KW�>�l����U���4+��'|Q�/��ht�)`[E�"h��nC0=+�+�|Kg��Ki�WI��1��Ouhe����Z�:��e�Z�b��e\v���+����#R�u��������h���Op�/|��
�����Z���?i�n�Sq����O�
���|['�o0�iic&�`H�|�~�4��<A{��i���!W*��(Uxr8�����N�����\�|5����I�Dl�`��y���y������m��?�kW�x���> ��O]=���H����I��:���?�H�\)�x�@�n���e����y>�q��{��i����Z�z{����e����;�^��>>���x��3xk�k>�}��w���
����<W�xB��^����W����9���'��[����}g��'��L�����4j�HO��G��zW����^s�x���%���.M��q��������-Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@x����/�><���^s2�����?��^�^K�����6i(��v��`�q�I1�'��?��@���'Z��xK���+��9f�����>s��*c��_�x�J�\��������wl���v�#8�T<���
x&�E��)���i��y�@���<�����6;�<�����c���hi�_�PA!io��^�
�#�?�:���������
��t��%����tY�x��d�dq����w\�-���#Z�R�
ock��<�8�s����1�5��E���/r��m<"�{�G��G�^�k<WP�n����2�H�#��^!�l<K���Z�o��\6
����A�W�X|7����^����IS���b������J�oq��Y'��[��1�T��d}F@��f�?���d�_h�����4�^���8����4�0���bG�^����x�W������	�]y�����*p�~^��~������p��X��q�+��5�r�=�2��h��Y
�*FA�S���<O�\�:�fKk���p�s��{y�7��?��
��gxc�6bB��e�_A����28�6v���^!��O�a�P�/�O{��Q\��V
��y9�#�����K�n��-2���]���ws e��8TmQ�^�+�����|!�����v�
6�O0�^���>�q�<c4�|S�D���|A��S\�!��:��2��$��{��?	����������{����G��������]�`�������
���3���:��i,p�w�$��r8��P�Y�$�@��A��u����x;M�.��Y�����9=3�m�k�~2��x�����]Ct��H��?�:�*�O�5�G���^����9�.b�s������F23�_��<|'��r��1$@�1H��	�+)"���Gj��z��'��R�����T��"���4����e-�rF]�2�n@��z'�494?i�5�E;���2��0po�y��x�\�-�����L�%�YEm�BC���z��[Z��������_n/��A"�5�@a��u�5fO�>#���su���XZ]6�4����������X�����N���C��K[H�Wxt�0�A�N�@�T���:���������<�
��h��2��A�"��W�m?��I�U�Ra�D��8���?^��Yx?���m�M��U���E������a�?�MzG�RI|3��L�i*�=vW�~�sB�
� K��U�	
F
���k�V�*��Y�����R1�(��=Z�/~k������[�N[�Z{��E��pF2N28����'�>��|�o��G|�������4[\������6-��\���H���mX|1�����_5�u/���6������@$�Oj��I���~�H��������!��<��Y���)��0��ez##�� I����>ARc�)!����i�����Q���+��KL�2\@ZH��>� U�|&����ko.��������q�y��{I���K�w���������o���I�����+C����_��r�����o<�D���(p�~S���G�����s�X\Co4���I�*�c��������08����-k������}0������_���	�^K[X�fL���RF{q\�x��2G��uo�5��o3;q���r��-��?��{��=��7M�eP�����_�T��W�31�rMr�>���t�[����b��q��g�#��P@5���o�`�x��a�Wu���&�rU���G^�S�}�9��D�D���?�!��c� ��m������c���6���U�@�_��������|Wl1.�~��t���?Z����s����j2:G-�*I!He'�"�8m����=sS?��������- ��B����[���
<Q��o��}c�Q/s����#����?�4�k��nn���,!��
�y����<G�~}o��^$���==��k�v
��1�`c�����:It)4o��������2��d�>����4|-�HH���(C������2�<���o�>#���w�|5� ���m��^FZ5>�@��B4�,G��u�����H����?���/�Q��,����56���Z�<Um����Z��dk;t�
�'�C���'�WE�/�7~5�k�}�6�4� y���������T����U��;|����l]x��PZ\�����G:����`0����.��t�;I3�D��	U#��G���G�<5���[��s�L����r.�'�q?
���x����2�K]^�R�����I�w�����y�{us�0�n��]��V<�����yO�
U�&��mJ:o�u]N80�������s���=��@U���i�r-F�����8��M�����Zv��{�'���������S�G(z��x����|}���x&�F��(&�U$��`y�=���/��(��}>��/�Cc����
���R�~�t��Im-�'x�����s�����!������l���a���=1��<y�_����m���&�-��������l�P2:����]G�+���P���?���?����0y��c�Z6�K�G��>���r2+��A��l�m�4y�Q�q5�_���������>	���������:<�m��%��rIj�~�?�xq��q��Wu��Z�,nO�5�5���D��mX�mQ�����x���5�&�uuo,:��dQ�r
������2�?�-�������xqB�WJ
0�8��X�<#w�o�������$��W�N{�G��=��gi#+=��D��	U#��-�������@������S����#����g���[���j!�
�)�C'�c�g�_�w�5\�ri����N���N�S��`���@������;H�����[�S����8gh�{���Z��Akq��b+�F�k)L��p�����}��x�k=OK�m/\��m��d��ls��=�z�����J�E��W��Sa��	��e��	�W�Ey�A�3P�M����K��G�q-�p��+7����<��o�
�V�������y���O���>���^��hV^��t�.3��m\��=K������<Y�����%xul�o�����o�=+�]��S�,�old��'�d��F����8#�\zx�~57���������0�f|�������J��
��x�����H+����|	��`��[-��Y�A��
�S�1�����O�����|i�YhV����m��V�����rr>�k����E���#H�m�2G�6���'&�f�������.t{���gFL	Q����w������
��@�;���m.�h���9�,a������:D�W�lt��G���`gL�$.23�+�|=���'�!��@�>�k��c���'�u��
��~�������6���!$���1������t_�k�o���}�;Ir��y��b���Wu����*�(|u�qa�6�P�G���Zn�k����|K
��b8�_�P8�5��,��n4/i�~��]*LG�FNF}����[�������7�E�^@��#�����@?�yg�M|o�xO�ZL���/�+�"#�'�����5���.�u���y
A����
>�>�����[������B�K��@��w�&��5��X����e��<����l9��LP������V_x�H�T�K�� �����
��o��W]��Ox��(5�(�����.pN;���
g�������G�<`O-����z�������2�jZ������6���0:��^I>�PmE��{]j�A��Ca��_.y�z��3���dP��N����_���F��a^)�|.���������.[�<��_!��s�.>���7K���wt�1�l�8^5%�;
6y'��(C�/�|�����p>XaS������+���K���������m7Oa���U�����~c������x�����j�)kn�-m���FG-��i������o�����J��C����������O������"���������aR���P�=��<+g��hI�uu
@;��)\��q:���b���%�}���]��nm�B�HO'��y��P�c���|y�Egog�i�%���o��m��s��������w�|<��
E���n<�=\�Wq�8��\=�����������L��������s�zW����^�occ���5�$��`P���[�N�F����-{p������a����v��S@Cy��u���(���5?����B�G��-������.q�H�t������_�o���v�6�$��c���8�z�b��E��^�4�Yc�k�g�L�R{�s@���h�����Z���v���r.���-���l~�OZ���v���
���f�y���� v��n3�z��_�>:�<I��^��d�����xY����T��������/����N����$�+��U�eRA�;��������������m�r���2{�v?
���o��C�/��#��H���������c^��������Fc��\.��bNK���y�����i��q���u���W����zW���&��(7����n�pNq����Z��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(����1]�� �y �)�G����_<?5��M+�����3~�������(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���� �X�����8������(��(��(��(��(��(��(����.��i�c��C{l��2n\���Q@
�5�%�1�@G��(��(�
z�������M"�uD��F2�y�|�DF���e�y�q��uz��x3��?��$��#����=���IH���x��EQEQEQEs^2��������g����>�Hq�0:��<�h�������^�Ey��~+nY�n��;�i���������(��(��(��(��(��(��(��(��(���� ���
����A0��k�`�����~
�j;a��o�j6@���
�I�z��.�7��&���{k{�xJ����{���S�I5���G������
�(�I�����8���+�����>���>R��_������OEQEQEQEQE�x�����������7��n$.���\��:(���������[�O����?���g����%���v����+2��>�RG�@���w��'��d��8�h��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(?I���
9�H���g��/�����SZQ@Q@Q@Q@Q@Q@Q@Q@Td��<~��-�w����'��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��� ;�����@5�|��+����'��z�u��������;���_��?�k�isv�L���V�p�5�������������:]CX�4�/�SP���3���{����2?:��Rv��<�C�}c���y������*��&���J������?�0���W�E��V�W����b�-�������(�������UO�L<5�C�����T�a���4������������.�~v��>����<�C�}c���W��
�� ���6/�*�*'Np��At�>v��>����<�C�}c���z�5��V�����:��r4v���Td�����|�C�}c���y������*=^����q�j�4v��#.FN����Y#Y��C@��C�}c���y������*�h�Q�������Q�j��������o��[�w���r�oh�� ���;���v��>����<�C�}c���r7F�:0f�j��������Gik����A�}�<�C�}c���y������+'C����
HXXKt��"-�����1����t��OS�����GQ�`����,�����y������(�������U}��o���4�^K6��x��H?)��W����_Z�O����m'�(x�2�0{���������;P��X���f�x�C��}��n._�R�K0�2�`p��W��!hZ�t��{%����le�z�Mmy������(�������Ux�D�D����3\[��o4M����p=�v�(�������(�������U�(��������;P��X���^��(�������)��n��."��p�a�V�������m@���(�������>Q����~������i#�\�m�=G8�4QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQES%����������&�p	R2O�6�x������n��9�j+U)g
��j	 �R�EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPPbo�y�|�������������I]���u�Y��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��� ;�����@5�|��+����'��z�u��������;���_��?�k�`�������^�Q��}.���{��rO��?�yG�����
���9����E���D]����x��OOJ��N�kJu$��6��|,��n
#�,�����J�A��ro���G�)_�?�.M�������L��������'��3Fk���+�������?�]��~ ���7�������S�?���/�I�8h��s�;W�|r�� ��O�&���|?�]}�_�q�+���jV����������y�����|o���[�������C�����@�������P�|s���+V��O�����D���ry��~~u������}V�������n������P�O��jx���N�4�.���{!uu�4>l���k�I'��������8�4�"���L�#�YT�� t�G]�5�?���|;g������O�AKCF<nt=h'��6x�-�������[�����ki��2�<q����{j�;��o�]���I$P��+G~A���C��A���_�W��Q�Mqw�[�UL�u����5e��[_h�~��I���-c�Q�n$r���%s��b�%��E�|���2j�?��P��2ya������V'���:O��N����k	,J�����*e��P��9�Z:O�u{��3xQ�V[�.�y+�,����xo���c��0���<+�	�5�C���\A���c��=���&��:��cc+�R��c,�2=3^�X����m4�X�X�Yn�a2����f�*�-��^�6���	jzm����4�$�G�H<f���b�[�^���_hR0K6a�.���F;Pv>+�����vZx��W������A�+�:n�������H7��������c1���7��P��x���6��moG:������	,�2���H
�M[��O������/PIney��S����;C(?�/��A�]n��c�+������5";/YMgp��Vxe��c�M��������&����{�[�}V�e�H��.�w6�.���G#���@�|5o��g��S����H�L�$��� ��*��@������
x�u"M{\���V�8Y"_���V���^io���[�a�E�G�Y���{�����b�|E�Q�Y���5��f��������y��5CT��7���{O����g>��6������_�w�k_��-Z�M�=,[������G�&R��`:`Ez�y���x�����:O���m,P[�p��3��-� (��TQEQEU��3m�\��U��3m�\��/QEQEX�i#���1�dsVj�����;��)�������EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPL��;AS���>�6�!��n��zb�h��p�6��
�dc�MP�l�>V��Z����55QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE[bi���$����sVk+R����9umB���Tk��4ns�I��m-��Z�V��������2�����	��?�e�?�:?�����Hw7h�/�N<)�C.��������xS��]#���9��=�;��V�'���H�������N<)�C.�����������+��
����t�G�'���H������x�i����	��?�e�?�:?����
����t�G<{���sv��������2�����	��?�e�?�:?���=��C��Ea�q�O�t����kON�,5{cq�^������U�C� ��~t)E����-QEEQ@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���������;���_��?�k�c���������
q�����w�����^�;������:�TS�C������`zQE�`zQE�QEQU.�[�KK��a��b��;���2B��n���jvZ>�%��u��@&��������*��2U�A�-Q@UkZ����q�j�,��t�g�3��	=h����2�������h��~�}oeg�P��3@(�z^���v{�^Cyj����RA����~1���t-�msO��8���Uo�4�EEos
��\Z���2�F����#�d\���6z��u���Cy�i��P0>��nQT�McN�t����`��R�g
��>�=�����WVs$�L���6��������|E��wV�^�kw=�m�8eb9#NA�'��Ep�K��)%F(��J ����:z*+k�/-���h��A�$��+PGZ��
(��
�7��m���U��7��m���P�(��
(��+O�$]�?��>z�1Vj���&K�����s�q��@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@2bXdm9�S���9�8Rp{�@�*�p�]�cR9��J���m�p�P��S��J��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��^�[ch6��p�ji6����U��Es?������h�������?�Z{)}fMEs?������h�������?�G��}fMEs?������h�������?�G��}fMEs?������h�������?�G��}fMEs?������h�������?�G��}fMEs?������h�������?�G��}fM^'�E���8Iw^?,���~��/����yG�;���4���1� 3�:�_��l]9F��9�5�:2H��(��t��(��(��(��(��(�����ZY��;��1�$ m'���W�U��	?�M���d��R��Q��27�VT����4��������w���������+���;#��B�e�|���sN����|7�Q��������+��������B�e�|���sN����|7�Q��������+��������B�e�|���sN����|7�Q��������+��������B�e�|���sN����|7�Q��������+��������B�e�|���sN����|7�Q��������+��������B�e�|���sN����|7�U�{������$�pG�����?�o�]|D����X\\�T��[tQEw�QEQEQEQEQEQEQEQEQEQFh��(��3EQER��}�^�������E|;�\d���]��� ;�����@5�|��+����'��z��C������������*��rO��?�S�RJm�����;T�t�)�H������������sy���R}���}���f���������]��ZD{����}77����5'�n?��o��h�-�����~�Ai�l����x�q\_�n2?����f�J��C�
��
�O���:O��}r�<P�\3,4�v
���W���K;���^���ib����$q�,`��X�>��v���Z��}R���W�iQ���<H0���������.�5
7��,�u�����k�N]���0� ��g��M�r����f�oco5��"�gf�����X�+���|]�x�����:ki����[���A���7^�������5���;�_�:��[����[aILoGM����y���O�j<E�M9U������D��\m'�O����������F��iKh��2s�c���+;m(�V^$�%���9�fu���\Er�������@=1��������U��L�G�,�����7�ns�\���<om��S���WV�q*��o�����Hq�N��G������MK6��m�	.���a�9��� N�7��Q�x������^���x����6U����h��
��$��Y�u[X�0"�>�OV���lB���k}�rZ�D$H��x��[���3[\k�\ws��Y�����w5&��Yx;G��4�B��oZ}����$d�=�T�� :����j����=a��B�M����k��bR_�K1����_���Yh�xz�G�-n���o6}>H�
��2�`��S�������D�t�r/�83���4�R��,�!�� 
_�����nm8Ig��j��!yP�(�3��W�k�~�>��_��q������Ko<��l��W\��1^�c�[���:���i�m����Ok���C�8���������������5;[v�� �2Ap�#w�>\6rs�@x�N��>�V��^K�$���W`���3�wI��O�<O��������7Y#��o����j����_K��h��Kq
��I
e(<��x�
�]v���
w��~���v�Y���n�lZ"Nx�����r�
�S��x�KrZh,��O/+G���bk/�>*�4}U��t����.��t�.�~�����5��~��/�/���d��!�	!�������\����	������� 3��s2�L�����q�({������7���P�{[h�-�c�g����k��k�j0�x�Y�,$�!�5�[YK���q���[�+���(����!�o���z���!�o���z�(��(�K��Q�'�$����~����I?�����A��v�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQES%,!r�{i���d��.�*q�9�6������0����q�KQ[+��K)���s��y�h��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�o���?���V�cx��=a��������k
��Q]��QEQEQEQEQE�?�}�J'��d��l�J�����E���������<w�����u>y5Q_*y�EPEPEPEPEP^����D���O�+�+����/�~����44w4QEQAEPEPEPEPEP]o��������J���?�o�]�?���C/�7���:(��s�
(��
(��
(��
(��
(��
(��
(��
(���������H-�yJ/V�	����8�����������A�j��Os ��/��#�?u}�R�
}w}��M��\]J������6��3k��W��O]W\�,z��!�����M���!�(���rj��O��x:
��#Qf��`"-�x��g��]�����z]O�:���m/HK�-��=.�������1'���������5�x����D{k�����oq�>��[�}�O����%���r����<�T^?i���_q�:e�0��8��h��z���k�.��i^�����d�K\J�}��?���t��j��MW�:��6�6�}�^\������{�{�����I�E��\�������
��&���jx|�~������s��}��.��>%|N�t(5��#��T�o��<�n���`�O~%����������OV����N�����_|����6����Q�7�o�sq9���=v>��������+�6�s$�����9���$���m�U��j�ZF��Z-��=���flm�_�G#����%�>|N�����_�� W��<�mn��*r��.��/��{y��{[y��t�~t���\�G�og3�\�mv&���=#Y����{�������
W����O����?����i?�^C����:����?x�NY�r-���H���>PP��=z���������:�Y��uY#3K)/�!����no�h�S1���A�2?}�|��~���?L����������77�@����no�j9���|�znp3���TQ��t3����������)�p~����:u��.77�5Z�����K+��kiL3�cq�[��tSJ�3���6w7��E4.w7�5^{�;;�{{������A���!A�@���E7oy����X�-$�TrX�(J*$d�E.��V4J�����F�2��hZ*'x�O6IB�\���*�$��2 �@���X�w�9����	_��Fo-�7��2�A��TS���v ��+��a���������mW6����L���'V�3}��@Ut�(�&��|�8�P��D����������q@(����Di���v������.��i�]�Vo�2�4�z�B�gs����	-��H��S+��H����f?h�~�63�����=]-J;7�gm��3��hV7_��6���_�f���C����q���p������D�����?0a���@(��h_g�M��\|�>os�ZV�-8���v�OE@��g2}�f���������������|����b����bd�M��G�Xd}8�6�������������,�U���"���v�eXe��R��y��:��X`��b�����o�&�s�p�������i���0���@(��hQ~�p��2�2�N)E��Li����p�=�t�Uv�-��n�~`�'��Z���p�F>V��(�[�I���|����B������$�����@�U��)��&��|�8�P��bd�M��G�Xd}8�Ush|��n8l��7n�)��"���v�eXe��P�*����[�3�V8�)~�~����lg;7
���'����I��;d��`~��hQ~�p��2�2�N(�XZ$����Nw���JV�-��n�~`�'��X����p��7�q�����iZ���O�L���OE@��g2}�b2N��o�������������q�8�@(��hV'O��������O�$G���������N�f�����E�M���a���J���[�3������EA�c��7�c9��m��H��$f�D��~Va����,QU���:����e�e~�P-�����Nwn�(�Y�	�S�7i?0a�������I�]�+}�b����N$�D�|���?
������v�OEW�������p����q�hV'O�������b���C����q�g~�����#���7��Xe��P�*����[�3���=x��1�G�������6�*�����$-��� �����KB������1�a��q@h����Li���v��{t�6���������0���@(��h\ �M��\eX|�����lZa'�&\c�6���'��[b�_�LFI�Xm�T������r��0��q��,QU���N�i�;��_�}��>_�n>�w���:P�*�Z��7�}��-����K�����@V?^(��&���D��vn,R%�IK��v�~Va����,S&�q�2�d���KB������1�a��Sd�����S���,	J��BY����F������V��� H�����Xc�R��p��7�c�a����,QP5�i��h�q��0�q�P-���~�1'aa��P�Ux�
�I��>f{�)��N�i�;��_�f���C��i�����n�t�H���E�M���2�^(�^KR��\��@V>��SlM������vn�OEWKR���D�������KB���7
�c,�+���,�U����M�����#��)�&O��
����O���EV{B���7�q�a�{�:�����O�N���m?�OE@-���~�1���o�����o���r��0��q��,f����ct�M�������8�������q���p�����,�U���h�i�]������Y-K�����@V��(��&���D�g;
��)�����;g?+0���@(��hQ\}���e�e}�B!d�M�����#��4UchL*�i�$�2J����M�������f�����}�u�>P��(�"���D�g;
��(z*�v���gm��3=��hV7O��6�|��W��Y�����>_�n>�w���:R5�h�>�p�s�o�f��%������h
��s�)�&���D�g;
���'����YK��v�~Va��R%�Eq����1�a���(�XZ'�nH;�����
��n����O���,QU�����M�����{�++�z���4)��V��Z[���`������9�����/0n���W����b�����3���v��T���B����p#
n�{���o�,��|K�3���z���@x-ct��������W��Q�
��'�������������G�g�>%���s��� o��m'�������r�_��?h�S��K���^}��G�g�>%���s���!?������k@g;����P����c'��6s������Q������}��a���+����Z+�?[m�ku�}��@��<!d�������r>�5�Y�������_�����0�y�����n�?���@x-��������������3�����������?�����J����|�����>U�\�����'A���������y��1@����"�YN>���iIEn�3�R
&k�L�O��>��FB��9�rzW9�YQL���y��9�4�P<������)�2q��������(!�Sd6w��'�A������dIE3�;H����Q���y���y�,��Q����0;��2[;�{@Y�2�������Nes��<���+a��{}3�y��h�4��J�vbd��y������������#<���%������@����k�y�����/�m0o�����=k�,��}�r�m/�'�O��0|a���4E|���M�[.���a�J,����R�������[�<���9�K�{e�Y�!~�P�8��z,���r�n[���"��=���n�<�����O���������e�v�r�l_��i~�hC(YC�{���G~��#G����H_a�J,�����EZ��c��6v�[?;�,3��n���?#����hC���������������/�~����y��$h���6�w"[�=��~J.|%3�Q���������9��K�I.�kE7a�����2�on;��;.���:�iL��a�CF��;���(��]�QM	���>��&3�����E�p��:�f�������o�h��]��M)�>v�=h)���=����YwE7g������'{���,�������O����"�F�9����hV�MW��s#��#��:Wf���w�������*�Z��7n~`�'��+����:��Xs�x�X�T
lZq'�& �6�*���O�L���F
��������6�����q�Z���>�pw�N(�X���7��2}�P����7�c*�-���,�U�����h�p��0q�U�(��(��(��(���X���) ��OW+������#G������,�dhPa�8�&��(�����F���_�L�,Z��u������Jz�d������o����r�:��1����a�����s]M�����V����"�'��w���Ml'�vnF��Mix'��������I�������d��>��J�(�����|X<S��o�YxD%�Ct���=�����|�	�Mz��~#��[��#���E1���A�}s�Q@O�<6��Y�����h���f�	c�3�!�?��"�����o��[:����-�@!��]�;�_s�]�KY����{����������q��F�v:�������O���?�H������5��O�I�\G��-���cB}�S�������yJo�����2:�$������:���8����s���������91��k��<�����e�?�%�������uO�	j�r=G�FG������s���N����`����]S�Z��X>/���T����w#�~tdz���y�iS�:4������uN��^Z�����1yxw�������O����������o�+��(Yi����m��o=��WT�aX��:����|I����y�[-J�+�i��HfP��������%�����Z
�����	��ac/��5\�`�;��]
��<_�����xN�=* �����xS����kSS�-�5�
]������C��Tfp�t�J�
W�[��2���_��dZ:{�,2�������s�`�9���f��O��1��m>���t�y�b���W���j���~������4������u����te\���+��$����Y�=>��k�B�G��q�l��Y���k�Xh>,���Z���Q5���1�m���C��4]S��qG�xr���P���P�Z�@e*3���OZ�<+�k�� �����h�	��4��uq�C8r%�`���t��^��,4}Qo.������N�>N������W��1�#���i�>9���+�����I�I,:������GP{}k&O��-,��x<.���2����3�������k�G���~'iZy��V�b�`�@�q���4��Ptx�\K�Ics��qv&Y�(@�0�q�
z&����,4��+F����?����]�����k��5mb��G�l�E�&�w������C*��� �w�X��������Z�N�����Q��5�r��v,�y_[�f��o��	�����Ay6oH���~E'�98��z�j����f��3S��Ck�Y�)V�5�M!���t���

�����$�LR��U:���f}�uu�����j�>�c������mt�v�������a�f�'��Z�~��Il���l[Md�-����O�pd����~+\?��;����b�}DQx��.�����+��J���7)��5�
Il��B�;;����*�e�H��\��<)��|!����:t�=��m�cwb��<�9'�@O��V�:��S�#�y��<���|�iz�����P����������*��a���\������������\d�={���>�x��V�7P���C:�Hk��q"m��s��8��.�>��X:�M���m{vO_6Wwa�d��|2>��p4Q��tS��y{��|g=+��p>;��i6|��s>�g�tG�-���=>�����X�,T�=�~����I�i�����H�C�f��;s�f����`3��)��-������>���X�>c���N;s^�@Q��C6�����Q��C6�����Q@Q@���M��[<c"��b�����n0����*�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE���>�J�9���L����O��B��,�Z����Fg*�5!Gn:T�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEW�|w��3���o�����������g�������1��8��H��>@��+���
(��
(��
(��
(��
(��
�o���1j�����1k�k������������?��e_E��=��(��I
(��
(��
(��
(��
(��
�/��������}��8�����_��?K`�+����c��<w�����u>y-Q_*y�EPEPEPEPEP^����D���O�+�+����/�~����44w4QEQAEPEPEPEPEP]o��������J���?�o�]�?���C/�7���:(��s�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��)k?����y?�\w���"���2����Y����{����������q��F�v	�!�?���um��Ly�����8���!�?���uz�"�,��?��Q�X?�����h�,E�X?�����������R�@X�������R�E��PJ�$t���
g���/����i���b�����8�TT\"��b��(�!%T���*��v�98M- `s�

����n98iH_qE-�G4�(`�� ��"��w���B�	�����dWu=�-�(����U�`z�3KFh������#�uf�"�,n#��L6��I�2NI�*J(���(�Fo���6��Fo���6��QEQEW.��*�F�v���V*%�z�w���n������
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
d��.Td�8�j}2]�K���N1��m�3�B�0���`��������/;>f��>�����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��������g�����u������?�[����O(������JDT��Q_�PQEQEQEQEQEW�|���V��%���^S^���F-[����F-x����*�/��tQE~2HQEQEQEQEQEW�|gPl4�'���\���+�k�~3������6�03���;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���������;���_��?�k�c���������
q�����w�����^�;������:����X�>��w����?��'��$��#��o�:�����s�d�I�r���E������C�z���oj������+���?�������4�������Q����U��b����T�f���?�����2?z�������}k��C9K���c��T������B���L
�x���c�^Amys:N��/8�9B
z5y���6
_�����|�����Z6�����=2����?�
���jW�:�S��O��k�@��H�����[S|J{[�c������_O
�:+Q�H���i>�um��k��Ae#G5��\Hp3�\�����Mg?��u{�����������)de�O%W!G9��(�����[G��2�-����_�w��0�@�~��=O
�kvO��x^Mk��m��o)-f�$�����	���\���H<3�����8b��ies�\�����W�%�@���{�K�+K�[�"m��*����oh�/���8mo|)����	���4)��k��q���|+�f�����h����k���Y"��8��� �g'��!������U��[�L1�p��7y�\��dQ�����_�'��@���3_����M%�6���.�c�2}�q��~(����m��lh}�
#��=6n���q���?���?|Q��Q���U���z���@� �K��?�xB;��$�1Y�m��	�(�E���e+��F(�������wEf��PuKk5�%���zc���x��_<mw�^�7�_M#�^"������J��
?T�N���>_���@����+��Ft_-��p�7���3Q����M>����fy^4G
Xq����f�=?�y����xgX�^ ������,��|�U����'>��I�x���y�����$��3�7��{���������7��n�� 2�����W�	'���������������p����G����<Y���gn��������?��F1��|=�X��~��^[� �����
����c�C����V��~�����k���y�`�����	�����_�<O�	�C��#A��g��Q��@y}�O��X:��^��5?x�R��[��#�<���oQ����B�9��+����\����S41�qhy����8���+��}���9��-O�2Y�k�xWO}>P��np\K�Rs���u�w�V��{soq���G��+B�X��9�����?�z{h4w~$hml�
��n:P���3_��<������_"��S�:���`��J��n_Go1Is;g����������?g-fs}u%�"�R��c �+wg=�6�B�t����Zx�V���OK���^IwM��[1-�1������5�u��?��U����6������,��NIM�����`����Z������[L��c��f��I���A<v����C6���������|:�o5��}-�24����}���5�7��m���P�(��
(��+�o�}�h�����#�U��Q�����S��29�4QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQES&�p	R'��L��;AS���6�JZB�C�A �<T�
�������F�0F:��QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEW�|w��3���o�����������g�������1��8��H��>@��+���
(��
(��
(��
(��
(��
�o���1j�����1k�k������������?��e_E��=��(��I
(��
(��
(��
(��
(��
�/�����re�����G�^�^e�����`.A��>��O�\x��y]L�|�Z(��T�B�(��(��(��(��(����'��������
W�W��$��6_����AJhh�h�����(��(��(��(��(����?�o�\�u���:����0���_�o��tQEz��QEQEQEQEQEQEQEQEQEQEQEQEQER��}�^�������E|;�\d���]��� ;�����@5�|��+����'��z��C�������\m��I6����S������Ec��r��9�?��?����I���j��e���C�|
?�,�����S?��?�����kO��������b}�;!���_����(�����H��L�������|?��?����Hvg��e���C�|
�T���?�����kO������QKd5�d]��:e�������3\Z1kyd����T����kZ�F��
��������L�����X��;�p������������OP�r�]�������|?��?����}'O����qco-�A�x�F����f���:F��^�igg#{x!TF'�J���S���?�����kO������
�o�t
����O��l�����z�j���ea%��Z�n�2��1 S+����}�?�kO�������Z��z7��hKM6��K�������3N��S+�������g\x;�Wz����
2k���%�'�8����������G�����o����Z}���Y���ql����0�G�J���
��Z5�����n"�U���\	f�kZ�F��
��������@kT���u��w��z}��P�[�ev
:��sV���?�����kO������4�hz,7iE��w@	���PJ#�s��u�~x,����O���[��������G�����o����4�54�+8 ��6%�q��W�/LU	x{K�7�n��Z\��Cj������������G�����o�����l��-����K�`����a�)�3��U/�kO�������Z��z7��h��:L:L�\Zm�XK���X@���W����t����5��4��S�u��~��O�kO�������Z��z7��h�Q��C6������?�����O�jq���Y�2O�h�EQEX��I��g����Vj�d��E���,C����f�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
d��~v�9���L���NG��M�c���O-v���T�
�V����S���:T�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEW�|w��3���o�����������g�������1��8��H��>@��+���
(��
(��
(��
(��
(��
�o���1j�����1k�k������������?��e_E��=��(��I
(��
(��
(��
(��
(��
�/������co���s����W���|f
�����Q+�c����<w�����u>y-Q_*y�EPEPEPEPEP^����D���O�+�+����/�~����44w4QEQAEPEPEPEPEP]o��������J���?�o�]�?���C/�7���:(��s�
(��
(��
(��
(��
(��
(��
(��
(���H�-y����Mq=���W3��+Mw�I������{Vo�����x�F����t
.�b�o&��8 ��?�������_���z����>_k(o�����t�-��D�=���2O5������9���$��$3��c*���9�
�$�R��_C�`����j�(���E�/[O��#�����0R2�??aZ^����uO5����%��:g�d.�.p��<�����@�Hh=+�N�>+|T���=�xw��-b��f�'���}��`�}=�m>|b��������,�����S'�fL�U�8 t�����Z�y���1\�R���/�vi=����?�L��b9������N
~*�:�������i�D�q�Y7��}}��5��������;���_��?�k�c���������
q���Y_��?�k�}I�KE&�P`Q�KE&�P`Q�KE&�P`Q�KE&�P`Q�KE&�P`Q�KE&�P`Q�KE&�P`Q�KE&.(��
(��
(��+?�d��^bf��G#��Y�K��P���N8���~�����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�L�`s�p�����L����}��w�2�����B��N�:T��;Z�f���-��q�K@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@y���#:�����=+���>;������������������*|��(���((��(��(��(��(��+������������)�V��#��^K���L��E�}���:(��$(��(��(��(��(��+��3.t�-�F�\c����������>����Y6��?�������������Q�3_*y�E�4QFh�Q�3@f��E��
����&��_��)^1�����&��_��)M
�QL���)�QEQE�(��(��+���������u�W[����������_��e�����EW�{�EPEPEPEPEPEPEPE�����F(��O�����+�iiy41��s#)E'8H�\��+��O������'HX��w|�&& 
�������n�o�/�/��l�;agI
@�dWG��i
���L����2���M�N�������,xd�:6����gywZ�f$�$��I5��	#]��8�����%��omb�n������i�������i�|I�A:;*�#"���/H�F��,r~����L�����3w�'�[}����<�-���p����i�v�?�#�R����K�v�0�v�O����(u?��
[�<;����k�nt���=9��/�O�l7Oo5���}'�y�^>��os�x��G�y�#i�-��kx�z������|RF��F�>���{)�g��)���t�*�qm�
r{L�/�-n(���X0���u�������/
^^jr������s��8iGE�W��P+�R4��3�6�k�YAu�@���|Pa�C���o�\&������$�8?������W{g��{w��6 ���b�dU=T��^�SP�W��������KT��u���~�~�1���Q������_����� �Q��@�~h���j��x\���\D�H��Wp;
��g�@w���'��k���$W����O���G��-y�Xs+��i~���~�n���[9��5f���0�I�6�q�X�<��c�Ky�(bV��Q@8���;������8�,�I����s�F#�l����+%�1��|�2�����c
���3kO�[?�j�X��a�3h9�l�y��!�Q[��J���Vh�
�e������$`8��(��\y�>���cc��*z(�X�3/������H�0�n��a�ec����EV�>O������l�����0�I�6�q�X}��Y��+�c�wy�"����+Y��c�~A�F���T�PgLe������8���1���0���V<~|U�(�����c��������
�&�6��������EV{][��J���J�QI v�2�c0{f�Q@8�������6?,��;cve�2�����j�Y,aH����ec�x��0�L_��[w���~��Y��+5�-F|���bV��4�X�&��g�01+?:�E@�q5����<H���8�l�I����s�F#�l����+��1���|�i��x���[V&�y�X�z���Vh�
�����A���g?\������a%a�y�4P,��E$&O-c��r����c��w��9�X����iO��$����l�'��Y�
�en��fXs#��"XB��/��9������@���b�������s��
cD��3jg����y�4Pw��M����v�J����V�������8����$��7�$�����8���1��|���V<~|U�(����<c����������a~�hm��[9����@��DV�0�������2H��fT1#��X�� �_h��}�������R%�Q���fXs#��X��+%�1����q��X�^(0�L_��[w���~��Y��+5�-�|��N1+g��,�0��w��
������Q@5�M8���<H�q��P�q,�a�y$�#��qS�@���=�|��m9����c
���3k�����Vh�
�����Cn�Z���9��!���J���Vh�
�e�+���H���K�8���|�������8���
�er3�������H��:���9������@��"�6�������hkZ$���P�?z���5f���0�7��.��Xq��Nk8�q)���F�l���� [8�s0���y�����lv0��o�����X���Vh�
�c
���3k�9����x��0�"/�m
��kg?\��(��B��7���������������$a���Pc��w��9�X����D��9���db9��X��+%�1����q��X�^(0�L_��NO�[9����@���c>f�$���<���2��|�h��8���P
g�%>f���0{g-�Kpf��O269��*z(�v0��o������-�+�<����V'��f���L"/�m
��kg?\�����Do3������@���GVo3*��O����n<��~s����g=],��B��e��db9��"X����f`�V?���EV0�L_��NO�[?�hkZ���T�?z���Vh�
�c����"����4�������:H�q��T�Pg����'��qI�1���|�T�V<~ub���0�O�6�3�X�=��a����y�v�����s��EV{^4F�0���s�+��#+7��H���V(�
�F�����?��qH�QG)�|����F#�l��(�XC:���9�����%�b�H��>nG�[9����&�q�2�d���*�`���2nU�V�q�9��{d��%a���>�vY���m�F��x�SPg�%>f���0{g8����������8���
���b�g�9����[R'�y�_��O��*�[�0�>W�6�������sC����7��J��y���@���FVo3*��O����n<��~s����g=],�IL��n9�db9��"X�����9������@��"�6�������cB��3j�G�[?�j�Y�a�P7��]�������q<�V�7t����8����Kq�
�������8����m�>e*s+�V(�
�c
F��3k�9�����a����y�v�����s��EVk^4C�a3�J��y��K(def�2��t�j����y�~���cc��)�(�2/���9����5b�������a�ec�x�X�!h����O�[?�j�X�BaX���I#����4=�2*�0������(��D�	[��1�H�q��Qg�y�~���cc��*z(�v0��W��dc���[R7A�m|g2�?�x�4Po����_���w���~��
c���f��1+����EW��YKy�P��:~4��#q��������8���
�eJd_3q�s#��i�����q��X�^*�XX�!h����'����4L+�6�$~������@���� ���?�5�|q�A��X��>`��;�dZ�j�����Fu�����zW����
	�<4EO��&6�r�"��|��x�`����������db6�m�y��
Q��y���_q�����n���go����@9�`�4l����8�����=�n3���g��P������W�e�����%������z0?
Hu{�u�cbF,����=q���U(���W��.���\���(�^`B�?�c?�,�����l�-v� n:rj���+��w��p�/�����4����m�/>��

����\�������1�T(��z+NU��}u���{� �B���0=��D:��J��3+nm��y����*�}^���}�v]�V�����C��g0!<�����|���Y� ���
�����X��O^��U���b�����b���Q����E_G�����%RH�?�A
� n������v��f3�]�~lg?x�T�v�F��� �dflgwx�����s�4�(���%
�g?�A
��w#�5%Y3#�Wi6�F�%v���~��(��9��H��`|�����s���>�,��g���s���5�����K	����p��_���zmy��f�����s+���C\x�0�����o��c&�s$���N��+�}F9�h����H��;�l�R����j����y���\MN�9��|��cv`B8���)"��!���m����dqU(��]��]�+�N���y^Ys����f�MBy-�
�lP�*9�j�s>��.��uK��2�Ncm����s���K��I��� �c��G4��<���r�
8�w��?�Lc��Q�s�:y9��6`C��G�S��iwyw-G��ED�V��s
��^H�P�����^^����v?��j�s>��.��u+��D+CB���5��{�	M$�w}���P�S���x�{?�O�e���?�����J]���]�y�s��bR�s�����O��+�]���9�x�����s��QG3���4F��s���(�3��SO��g�9��g��H����.�s�����G3���0��s��F�������E��sK��-w��?Zj	#<��us0�}��y�5��V���*6��9����|=� t�}��v`�u5�w�$�g~��ZkZ$���S8���}��,�0��w���J����+�=��������qB����������G>��OEV�����>u�s+?>([V&�y�X�z���Vh�
�����A���g��a�+y�A��X^j�]���Ev�2��=�V(��
(��
(��
(��
(��
(��
(��
(��
(��
(��)k?����y?�\w���"���2����Y����{����������q��F�w�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEV*��H����0��#����L�I�y�S`��dU�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��)�`p�i��N)����>���s�LP-�8Dm�k���F:��
��g�O-v��T�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEW�|w��3���o�����������g�������1��8��H��>@��+���
(��
(��
(��
(��
(��
�o���1j�����1k�k������������?��e_E��=��(��I
(��
(��
(��
(��
(��
�/������T��9�p1�*���/��������}��p1�J������gS�g��E���QEQEQEQEQE��	?�M���d��R�b���'��������
SCGsEUQEQEQEQEQE��{�@���:�����������u���/��2��|�����+�=���(��(��(��(��(��(��(��(��(��(��(��(��(��� ;�����@5�|��+����'��z�u��������;���_��?�k�}EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPfu��D������*�W.��*�F���9��QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE��9#p
r~)���X\��
p1�@�`�p��E1�
;q����Y��q�1� `����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��������g�����u������?�[����O(������JDT��Q_�PQEQEQEQEQEW�|���V��%���^S^���F-[����F-x����*�/��tQE~2HQEQEQEQEQEW�|f��Y
�%pW� c�����k��3�6[��@z��!\x��y]L�|�Z(��T�B�(��(��(��(��(����'��������
W�W��$��6_����AJhh�h�����(��(��(��(��(����?�o�\�u���:����0���_�o��tQEz��QEQEQEQEQEQEQEQEQEQEQEQEQER��}�^�������E|;�\d���]��� ;�����@5�|��+����'��z����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��������>Q����~����F��W�6��m��r;U�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���.�e��mu8#i���mX���8��\���!K����U��J-\��&i�c��������q����������+�������>�����?���o���j��~?���
�h���>�>�����?���o���j��~?���
�h���>�>�����?���o���j��~?���
�h���>�>�����?���o���j��~?���
�h���>�>�����?���o����Yy/����q�7\}+����o����>�>�����X����7g���
����X���s�M\��QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE������?�[����C�<���$gY�z��G�zyG��h�?�R"����(�������(��(��(��(��(��[�/��Z��y/��Z���o���1j�����1k����YW�~hs��+��B�(��(��(��(��(����>���[����o�?�-zuy��v�JR92�A�L��+���/���O��KEW��hQEQEQEQEQEW��$��6_����AJ������&��_��)M
�QTPQEQEQEQEQEW[��������������O����f��#������#N�(�\�B�(��(��(��(��(��(��(��(��B�u }M-R�����
)	��.h���:�JzPE��  ���)�?����y?�\w���"���2����Y����{����������q��F�w�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEV(��h���%��rFG5f������68�EY��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�CS����������O�B���J�]��<�&QE2B�(��(��(��(������i���=;�if�@����@`:������	&�������pV8q�U��?����F�u������Y�����}�E|C�I�=��4y��G���_���/�?�~?�����|�?����F�2O����������C�O������+�2O���������z?��h����������z������z?��h�$����}>����?������>���!�$����}<�?����F���w��?�~?�����|�?����F�����-�x�������^���5�]�?���z)O���#�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
�����Fu�����zW���|w��3���o����<��F4?��)S�g�QE~�yAEPEPEPEPEP^���F-[����F-yMz��_��o��_���g��,���4��E��!EPEPEPEPEP^e������
H2�-��1���U���_�g�xo���|�c������w�������%��+�O4(��(��(��(��(��+����/�~����x�{?�O�e���?������(�((��(��(��(��(��+���������u�W[����������_��e�����EW�{�EPEPEPEPEPEPEPEP�_|�<��kIss<����z��\���0
�f|�g]C��L�������M:�y<������]��>�[�w\x�KK�����J)9�����{%����|=��7ZV�
=�����L�|��S��2���O���~-���]�5��i�Kp��m
����5�����D���<��O}�c����m�y:+7|1��~�����?�qaq�������������w����{{[k9t|���t�T��h�|`�o��_Kqq�������	;G)��`�e�~~����>
���x�\]i2Y.����8h�H�q����[}����4�-���;�������h�G��-o�.����a�,���|04L���[���,�����)k�S4i<��;�<�c��Sh0����v����������Z��Jd�,�����=��O��������e��p���*��7���~b���5�7����s��Os>?�N@��hz>#�c������~���{�kyLj����s�Oc�Q�9~�U�,t9���#/m%��4����n<g c��V�*F��Fx����{�(.����z���A����:e��p��^���4����F�=#Y����{�����������2����Y����{I����
<?��+�5�����/���'0����=����P�I�m`��O���>O������l�����*�XB�$g����%`<�K%�2-�|��Z�E@l�k�8����6?,��l�I���q�Y�}���EVKc���ec�x�XB��c���������EV6�DG��G�[?�h{dTV�0������*��E$�F�7.1�=�J,�w��9�X��������0����e�21���%�)���q��X�^(�[��>W�6�������sCX��$g����%`y��4f��%�2m��|�bF~t�g\	����cc��(z*��&2���=db9��)����_3�]�2�����,�U��+F<��A?�l�y��Ba~�h9�l�y�4Ug����� ���?�4�e�o3*����4b���q�;�����l~Y�$v0����e�21����*�XB��/������G�a��y�������s��EVkZ$���S8����id��M����m���X��k8��Lw�$lq��P�q$�Q�y�Y�}������1���|�i��x���laX�1�mb	��g��Y��L"/�m#����sC�C""������������a�<�!��dq��Y�[���a��"$0��0Gj����G�����6?,��'����1�2��`A��z�4�a
F��f`�V?���EV�>I���Kn�Z���9��ah�3�mL��y��4f���0��w��.��Xq��Nk8�q1��x��������l�I��~�I�F#�l���0�/�����X���@h��c
���3k�O�[<{��a	�E�����l����EV{][��J���J�P�"�y�P��:~4b���q}�����?��qH�P�#:��`A��z�4b�������q��X�^(0�L_��[w���~��Y��5�-�|��N1+g��,�0��w��
������,QP5�M8���<H�q��P�q,�a�y$�#��q@�U���=�|��m9����c
���3k�����@h������Cn�Z���9��!���J���@h��e�+���H���K�8���|�������8�	���er3�������H��:���9�����,�U��"�6�������hkZ$���P�?z���4f���0�7��.��Xq��Nk8�q1��x����8�	���%��7�$�db?,���1���|���V<~|P�*����<c����el���(�>H���Cn�Z���9�4Ug���Do301+��+��#�7��H���@(�>��<��~s����g�er3����21�f�8��f���<Cs�j�^�]�`��r��`���e��C��z���!���W��|_���f�f������2rH�~9��7f�n��*����}1���O�����i'yE7�uj���
�qJ���>�����7�����C�4�C��z���!��}5����u�7�l��z`~C���e1��r����l�>�����g�����bG������������h��[��?�_.����f����L9�s���iU���Z7�� l�63���S�7����?����H������������
�o������k�����������]����\~��Mf�K��o#|Y��t�Q���������/��>�#��h�[��?�G�4?�����������v.���<���:m����k7p�,��n����B8�`~��+l��/��>�#��h�[��?�G�4?��������������x��v�m���'�$qM���� �|���&�������+���/��>�#�?�h�[��?�G�4?���������e����H_��������N�Y��h�#tM�qn�g���G������/��>�#�[mb�����qskx�lE�kc�#�CR�9�2n~��K����F������aR�y��h��x�j�i�[&���s;7rJ*3
�3��iLJ[q��5���}�����4Us��1�������*�p�4y+�o8�~��P������@9���iLJ��?�4j���.����h(bFs���P�}!��0B�7s��4�!���5
.|���]R�H��N�lc'q���K�B�
j���������{c��jw0��<����0���#���5���$���N�����v���6s�T�z������f-����(n?��ihO����3V���&T�b6�aA������;�'�V�w������c�t-�<�����s��?�w����1��1���s�J�N�H-�P�=8�(�B��S�n-J�uO+���P�~��A�N�fW�����v?��h�B��U��I��K��V��1
����I�\�$n�Ncm��Ps����G��r�Kj�o  ���!S�R��s��U*?r����*K]N�jI(��H�����2:`Q������M���	W�����{P����~�f<JX�x$����Ns�5����3����w�|�O��C�z���oj������-wn�?Zj3��1������=�O�������?�����G����=W����(R9����.�s��������{z����������U��b���C�z����bR9������9����g��=�O�������?�����G����=W������v�s��F��3�������{z����������U��b���C�z����"P3����]�s�������{z����������U��b�i�����K"�v��X^R��}M[� I5Kem�������Y����4�ZnqNO���QU��'�J�f��bF�l��8���7���6?,����I�����1_3,9��_��l!H��������y��,�U����~�n���[9��45�/#y�S8����h�^Kee-�e@0����8���w���6?,��'����I)�|��9��G>��KcW�a�ec�x�4Uac��f� ���<�la0�g���#����4f�������g���wOa���fm���5�|q�A��X��>`��;��::����hP����DT��q�[��������
���3����.�5���lv���:g����y��*���^�*�����L�F�I+�t�P�G����>���3f���M��~�1�z9�[�_���E_�Y���Y���.7f�����RC�][�����n�F���GAG5o�_���E\]V�,M��|��y�`��������SZ-��>Z�� @x����V�
��}������kWs�M'���z��E���������K��o�J�-�Q�\���}��B�}u���{��y��Nm����f�	%x�����u�}�8�(�������+������������6�V�����������z�Fk�>^Kq��I�d6K�lj��b�H={���:���n������K�E0D�I���H!P�
�=~c_��N��T~J���3��h0�P9���5
	(��*��x������s���P�}���3��� �Fq�F�h�4$���T)^p�4y+�/8�5
	(��*@w�4�%-����h�4^e��7�~�A��_#x���zW���s���6���t�1�='bC|�8������������R���L���&�s$���N��+�P}F9�h���7"��	��JcLc����O6��S������<����Ln�(G��$Z��!�y_�b��<�L�(�B��T�eu	���+� �bR���4�j�l�7��@(<t��@�{�sFj�����'1���(9����Cj�/p�'z!@9��
���<�W�r�
8�w��?�Lc��Q�s�:y9��6`C��G��Z=�tU����h������T�z�Fh:����������T����3F��{�s^����D���O�+����&���!b����{�����R�>��ku� ���1OA�^�kE3�]���SA�J�����^�q�SJq��1������s��@�u���9��A��<��4
;����.�9��4yK��>����>�a�H��sJcRrs��4w��)�Z���~��Fy�4h�����O����"P���S�Z�&����df8r9�v`��>G~�����TUf���H���3�J���<��c�wy� ���8���=��Y��c�x �#c��([8�s0���y����8�	�������g��Nec���a
���3kO�[?�h�X�Ba~�h9�l�y��!�[��J���@h��e�+���@0����@Q@Q@Q@#�t*z��Z(����^�0�<IkfI�H���������|e<Z_�5����W�Ro���Y���t�P��+I�Ay�xg�:���u��lLs�������[��f���!���,�4�k����-���3w�����E��O�rx�A�����#����m�����'�>������
�������������}�N�����]=�x�����5�u�7T������~�`�#�����M����^������������|��ec�^�A�]m�x������,�8�n��b���l\,���������P�W��������KT��u���~�~�����(�EvtP-g�@w���'��k���$W����O����?����y?�\w���"���2�����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�
����U�^Q$��F9�Y��>W�A�vF8�f���(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��~9��f�?������������ms�����������E�?��%UO��Q^�EPEPEPEP@�(�u�/���':�z�	���o�����������W�8��������
(��(��(��(��(��)GN��i{u�@0��]J�E�	�/9�q��b��jWF,��Wn8���r��T�aET�(��(��(��(��*[o�����/�Km����E�b�N?���x�R����x�UQEQEQEQEQEsI��������:����^����j��k����#���t�������(��(��(��(��(��������g�����u������?�[����O(������JDT��Q_�PQEQEQEQEQEW�|���V��%���^S^���F-[����F-x����*�/��tQE~2HQEQEQEQEQEW��g
lt�~��M��������?�}�J��d��l�J������gS�g�QE���QEQEQEQEQE��	?�M���d��R�b���'��������
SCGsEUQEQEQEQEQE��{�@���:�����������u���/��2��|�����+�=���(��(��(��(��(��(��(��(��(��(��(��(��(��� ;�����@5�|��+����'��z�u��������;���_��?�k�}EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPc�L�M��L6�����U����6��yL����5f�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��>=���%�\���":�����Y���a��#�?���������yU>6QEzDQ@Q@Q@Q@����P��?���_��?�&�������Nt/���WI_�������_�$(�����(��(��(��(�##)iNx>b�[~�v�J�������j���mR���c;�\�q��W��vyr��QE$�Q@Q@Q@Q@Km����E�b��m�������S���X��JW�X��J���(��(��(��(��(��i?�����W4�����
]?�z���$}Q��P:Q_@}@QEQEQEQEQEW�|w��3���o�����������g�������1��8��H��>@��+���
(��
(��
(��
(��
(��
�o���1j�����1k�k������������?��e_E��=��(��I
(��
(��
(��
(��
(��
�/���?K\�W9�0����6���1o��,�2�Nzp1�*������gS�g��E���QEQEQEQEQE��	?�M���d��R�b���'��������
SCGsEUQEQEQEQEQE��{�@���:�����������u���/��2��|�����+�=���(��(��(��(��(����E�/\�W7�|=�>��V�e�%Cb54���������?������$,��;M=V1��5$K�g�����L������~�t�'����@�EQ�u?���P�%����9��M�D{��U�(��(��(��(��(������������
q�����w�����^�g�@w���'��k���$W����O���QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEX��I��g��=��Vj�u��D�7�,C��28�4QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE���?�,����������|s������y�~�������#���������+� (��(��(��(��(E��>��y�$�B��A��5t���<��s�������J�'��W�R��!EW Q@Q@Q@Q@!�9�����9�zP��Q���w�`�����;�V�:���.���	�=���V�������0��*I
(��
(��
(��
(��
���?!�����ER���?����������4������4�E�Q@Q@Q@Q@Q@\��/m���5N�i?������5��H���(�t�������(��(��(��(��(�<���$gY�z��G�zy���#:�����=+��?�cC�q���>|�EW���QEQEQEQEQE���b�����b����|���V��%���^&�"����@{�Q_��QEQEQEQEQE�_�g�x��5�z����M�2�����Y
�%pW� c�������/���O��KEW��hQEQEQEQEQEW��$��6_����AJ������&��_��)M
�QTPQEQEQEQEQEW[��������������O����f��#������#N�(�\�B�(��(��(��(��(����K	��H#i�	?���/mR��{Y���F��`A�t���G�/�_��j�1��nf��)	
����<~uC�^6����4�#��ah��N�������18$���{`Y�?������'W]4x_K�I��9I����FS��?���&��k?<'{��+G�E��b����/^�9��v��J(��(��(��(��(��(��� ;�����@5�|��+����'��z�u��������;���_��?�k�}EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPs#h�mLLwm�9f�T��z�w���n������
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��>=���%�\���":�����Y���a��#�?���������yU>6QEzDQ@Q@Q@Q@����P��?���_��?�&�������Nt/���WI_�������_�$(�����(��(��(��(�'��s�={P������IeY�Ns��^�_��W^N<�9�c�7Uz��n�.[���*I
(��
(��
(��
(��
���?!�����ER���?����������4������4�E�Q@Q@Q@Q@Q@\��/m���5N�i?������5��H���(�t�������(��(��(��(��(�<���$gY�z��G�zy���#:�����=+��?�cC�q���>|�EW���QEQEQEQEQE���b�����b����|���V��%���^&�"����@{�Q_��QEQEQEQEQE�_�
-��K =r����k�~3������6�03���;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@W�.������2"y6��I��V*���,g���Sl��~P	<}y��|A���Ks���8�e�����Q��'$��].��?�z�����R�-�r�Y�l���A���\G����y���KM3M����.�e�Jdh���RB�����v�����i�	xx����~�����������@��FR��Z(�@��Z�u��I����S��d%����M�����j��wP�[��n�P�S��u���������[�1���������C������7��4�����
��w���]%���4Lc����7��������43�"�4��>���#pl�����
��u8���5�6���x�Q�<�&�vt�>�9�8��~F
F[?���<n<�kP� �����,F��-��F��c�:����=X�I�����K����u�;[���;��	c�\N�,�������I����_�V~��n�����omq)�Z���By����z�8�H# �W�x�)>"�c���Wh-���[��c�T����9�q�}j��G��h3\'��F�g5��4���:n<g#�����?����y?�\w���"���2����Y����{�����������2�����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�
��E_p�"`F����V(��H���
����j�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE|{���K6��l?�Du������k����DG^_��?�/����(��|l(����
(��
(��
(��
(��
QE(�(�O���Nt/���WI\����':�z�	����q��u�/��QErQEQEQEQERzu��##(�A��;�A�i��F07�^��,[T�f]��rT��1��f�.[����(���B�3Fh����(�4PEPR���?�����[c���t_�)��q��7������������
(��QH�(��(��(��i?�����W4�����
iO�^����Tv��Q_@}HQEQEQEQEQEW�|w��3���o�����������g�������1��8�h��>@��+���
(��
(��
(��
(��
(��
�o���1j�����1k�k������������?��e_E��=��(��I
(��
(��
(��
(��
(��
�/�����|�Y8��3��zmy��v#O���A��oL���W;��_�S:�<��3Fk�O4(�4f�
(��
(���3EQE��	?�M���d��R�c5��	?�M���d��R�;�(��AES��(��)QEQEW[��������������O����n��#������#N�(�\�B�(��(��(��(��(���C:]����/��q�����z~5j�ox�Y���p���n�<>J�53���7	b�>l�8�;���x���*�����uanH��p�,Q���z63�x�Wc�x�����ig�|>��-fr���r������M�����x��E	RhdF�j��h��I���m/R�=�8_�[��S��8G�@�EQ��R�G�}r �0g���F��	��Z��Y�#�;P��������x+��g�����g�Dz���`�>�����m���5�`�+�2��������?�v�i�-���H���i-��[��"��O,�$�5�xO�Z��%�j�)����kw/�������Z>��K����iw�m���FbKH�$�k�~D�'���kGf:%���[�[y� ���kS�N�n��M���[k.�*9U�FEu�V���%��h�&g�n.'}����f�@O���&���?�ui�M$�z����YO���I��a��G�	v8S����G��
[�M�
Y�<=�������.t���G�V��|	��:;�`���Q�2�P�}�N��{��|4"����]2�����;�S��pH#����x��h���	��;��������3��WK���6�"���4�J�C�����<�u<0�|������z������|�j7�FQ�@*��zP+��4��O�v�kjt�x\�\s�R�T"��7��.�����eDq�I��]W�~�����P[��+W����l\,�U9���������-t
r]{S�/��nH��{~�����(�h���}�^�����G�?k�4MJ��K��'T��8����v���������O���?�H������5���O�(Z$�6��V'�|���'��������s��j�Y�!x�3�mL��?�y���
��>P��?�X�� 6q5��w��?��qH�Q$�U�����G>��X��+%�1��|�2���P,!X�1�mb	��g��Y��+L"#�mS��������� ���?�5f���P�(���\c0{f�Y�.<��~s����g=],a�����dc���KR7E�0�2��s�Y��+}�'��������s��hkZ$���S8���}��Y��+�c�7y�"����+Y��c�x ��l~Y�OE@�q$�U�����G>��5,a�\/�����X���Vh�
�c
���3kO�[?�h6�D_��G�[9����@��DV�0�������2H��P��:{f�Q@}�!q�|�������8�����|�� �F=�EVKR7E�0��X�^(�"�6�������sVh�
�cD��3jg����K%�2m��|�h��8���Y��c�x �#c�l����'2���z��s��T�Pd��0�|��m9���c
���3kO�[?�j�X�Ba~�h;��[9�����DEo301+��Y��(Iim&�s'�"8�{������;�����l~Y�4���A���K��b��t��7f_3,9��_��,!H��������Vh�
�`��1~�im��[9������H���3�J���<��(���&��g��F$a���9��i��~�A�F��*z(��'2������G>��6;c���i��x���4Pe��bh����'���=�A���"�������s��Y��+=�.���a%a�y�{(d�]���bF?�EA�8���|�������8�K(c��w��21����(�XCn��a�ec�x�X�!1~�im��[9����@���c>f�$���y��,�0��w��
�0���P
gN&;��68��([8�s0���y����qS�@���=�|��m9���"���4c��������4Pcc	�E�����l������ ���?�5f���P�"�o��$a����_h��}�������T�Pt��9���dc���KQW�������Vh�
��	���Kn�Z���45�-�|��I�l����EV��n�>E�1+?>i�g�%m���=�����l�Y��~�I�F#��)�����g��Ndc��Vh�
�c
���3k�9����x���"/�m
��kg?\��(��B��7���������������$a���Pc��w��9�X����D��9�~�����j��'����>/�p��e�B���X��9$e��u��t���
�t�>���]��?�,����������YUO/��W���<��������s$��o���B8���(�Y���c�?z��u�7>�}P����e}�]�c�n��kT�|������c4����b-���b
�O��j���+��o}���������I������t^G�?CK&�w%�S���,�������T(��z+����]���v.���<���:cLc>�E��C<����b����`~B�>�F��_p]������x��v�m�#����4j�"��������P����3��:(��~U�`�.���Mn����#l9&�&�y4���F��r�������P:�>�G�W��.���M���I~����h�3�����*Tp?�5��<��s�������J�;�UR�i~l�fFaV�s����)�Kn9����ErY������4Us��1��>�,���B�H���h�Wn�q���RQE�s23
��>cJbRA9���O�� �c<������h(bFr�4�(��1
�@�=~cG��
��O_�����<��Y3>l�����o$"+H���V���j�K�ZB�Vdm��P����3Q5K�#q;�`1���j��)_s���}�Q�7�����bRy��3Cj���>_�D����3Uh�����.��u;���X��.!A��i_T��x�o'|y��P}F0j�sK�s��s�N��&�o+��JcLc��=R�)��<���[0�zq�U:(��p��r�Z�����b�t(y?Q� �'[?�/����R�����U��g�9���&�q,�[�b����t����F��f6���>��5N�9��9���uK��Y���T���c��-u;��$���#"��S��1�Y�-��~C�]��9��|����O�#�	�HcR�9�����c�i*��r���o���s��5�y�i�Q���4��(R9����.���9�i�Q���4��1)��iLjH�x�4�(�}��]��k�w9��#P��������3�0D�g�sG��q�>��E��sK��-v��>����$�����n�GcU����!{o�����'��M)J^�:�:�����������q�8o����l~Y�N:Q^���t��6b�fXs#��"����3k�9�����Y��+}�'��������s��hk^4F�6�q�X}��Y��+�c�����bF?Sg�����������T�Pt��%2.��9��G>��KcW�a�ec�x�4Paa��f� ���<�la0�g���#����5f�������a��Xq��\��	u����!~c�`������F�<���$gY�z��G�zYJO0�����DT��cj�/b-���(��g��M�]�,k'���:��E�t�?CTh��}�+��}��v_�Y���)���	%1n�s���h����g�q�l�������3���QG��������z��W��F�q�6�G��$:������8��6��=q���U(���W��.���\���O�T�0�l�����]Z�kE�'�]��95J�~��������]����'x�O#1>��n��|~���n��.��F�W��0j���E}���}u���{��y��Nm����j�	%x�����u�}�8�*�}^���}�v]�V��������vs�^H�z�����5�B	6l��vm�T��Q�'�z�*�o���1j�����1k����YmV���O4v=�D��3��F�B�H������v��f3�]�y�s��*�O�����9��
�3�>���R��������.��6����i*��y������ �d~J�+��F�%v����RQE�s23
��>cJbR�9����EA��yK�w9�x��|ag��K�����1np@�������_�g�d(��1��c��u���w�����|��5K�&�V��FI\B�s�1��G����D���g���>���T�����s���r�j�1�$��o��
����I�sp�W��f<�L�*�sK�s��iu	���/� �b\����&�q-���V�D����f��G3���\�T���/��6���>�����\$������P}��T���]��]��T�
8����?�Lc��Q�s�:y9��6aC��G�S��iwyw-G��ED�V�'9�I��$f��Nl��|�/n��K��c5V�9�p��r������?��#��{��{�����R�>��ku� ���1�^)^����D���O�)�>�R�s���n9�}M%*8��E>g��iwbS���������s��:�9�p��q�5
�s�4�%�y�4�(�}��]�yJ�p}�R��8��}s>��.�J@<{�S���������iw������@�A$g�sN��f����@�>���
�94EF��#�#��3]o���������n���������i�ah�3�mL��y��4�X�&��g�01#?:�Ez��@�q4�c�x �#c��([8�s0���y����8���
��C}�g��Nec��B�B�4c���������EV60�D_��G�[9�����DEo301+��Y��+��2H���(bF=�V(��
(��
1E�q���o6�H��O��l�2*��Z�#��8c4
�+���7�F�%��Wi���I�Kw�U�Q�F)���x�]��4�����r�,�R '�#�h���(��(��(��(��(��(��� ;�����@5�|��+����'��z�u��������;���_��?�k�}EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPc ��H�.LL���Gf����AP��'�;��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��������ms��������>9��f�?����������_C���Q�T��QE�QEQEQEQE��Q@R�<��s�������J���9�����M]%~��{��)~l���+��(��(��(��(������9:����)�]�b�gq�����U�����WK�	�)�x�j�|\�g�-�QE$�Q@Q@Q@Q@Km����E�b��m�������S���X��JW�X��J���(��(��(��(��(��i?�����W4�����
]?�z���$}Q��P:Q_@}@QEQEQEQEQEW�|w��3���o�����������g�������1��8��H��>@��+���
(��
(��
(��
(��
(��
�o���1j�����1k�k������������?��e_E��=��(��I
(��
(��
(��
(��
(��
�/���?Km�m��;����M�1����D��,�y����+���/���O��MEW��hQEQEQEQEQEW��$��6_����AJ������&��_��)M
�QTPQEQEQEQEQEW[��������������O����f��#������#N�(�\�B�(��(��(��(��(��j��{��axD�p*F��u�u��Wv�����)P����F��yw�w��7�������R��W�$��I c����������ohWw)og�i��!�G�l�z���"������m��isy	�,6v�Lb>�A�>��������Z�MO�:u�j-�9�$��}��"�=>�(��(��(��(��(��(�
Z�������O���?�H������5����}�^�������E|;�\d���@�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@����>O�A�vF?L��Y����I�x����y���EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEP�����k����DG^^����K6��l?�Du����S�"������*����(�H���(��(��(��(�uP:��������������W7���I���^��Bj�+��w��_�K�d�Q\�QEQEQEQE����B21@0��:���Y���c��^�j
�T�m�wL�i#�5^�.[����(���B�(��(��(��(���������1QT�����t_�)�����o��%+��o��%QaEPEPEPEPEPW4�����
S��O�����������Mi>��J((��>�(��(��(��(��(��+�>;��������^�^y���H�������J��������EO�� QE���Q@Q@Q@Q@Q@z��_��o��_���5���b�����b�����������EW�$�Q@Q@Q@Q@Q@y��r�a����Y6�03�+����364�-py��q����W;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@T�>���w����'��v���u��	f���{�j�hX�g������a�a��"�8�����������I�2�|�?q��<�;b�������	��7�
T�������{��|�q��3��Y>!���5I5+�~���c�i���&S��q�|WC����V�~���=:]F��,'2�9�n<�������(��(��(��(��(��(��� ;�����@5�|��+����'��z�u��������;���_��?�k�}EPEPEPEPEPEPEPEPEPEPEPEf���-�$����5�?����5�)�+���_���;z+�����}<����F����}k����#�����h����}=��}k����#�����h����}=��}k����#�����h����}=��}k����#�����h����}=��}k����#�����h����}=��}k������6���`FG9�5��}�jR�,_��'#��WIX�<��U9s��(�����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��~9��f�?������������ms�����������E�?��%UO��Q^�EPEPEPEP@�(�u�/���':�z�	���o�����������W�8��������
(��(��(��(��(��)Nx���=;���Q,���u��w%s����Vu���~n7���oL�=*�|\�g�/��QRHQEQEQEQET�����t_�**���?!�����0>�������������,(��(��(��(��(��*���!{o����usI�������������/�G���E���Q@Q@Q@Q@Q@y���#:�����=+���>;��������^�Q�#�������3�
(��~<���(��(��(��(��(�V��#��^K�����[�/��Z��y/��Z�3��U�_������d���(��(��(��(��(�2��[�?Kr�W��N?��^�^e��w�~��m�_w�p1�J������gS�g��E���QEQEQEQEQE��	?�M���d��R�b���'��������
SCGsEUQEQEQEQEQE��{�@���:�����������u���/��2��|�����+�=���(��(��(��(��(�}R�K�������Iw�Y��*$�gc>2A
>��+������=���j�d�[�Cgrc�R�������G����)�v����!�����f�~�l�l���������,�|-����u,Is�q����n\�N��s'�������x���8����R�O�`�����?4o��f][U�u��MJ���w������(��B�$����B�~��ih���^���9��B@'�A��-^)2G�S��}�^�������E|;�\d���]��� ;�����@5�|��+����'��z����(��(��(��(�o������o��x-�o;�Pc$�^c�
3������_�]zXL�����������dJq��g��^1�
3������_�]��>��>��E���_��������>��g��c�g�?���(���?��|3�@}[������^����?�=�;��Ex��4����~Q�t�L�g������G��������{Xw=�����i���V������������o�/�.��{5��/��0���{=���>��>��E����
3������_�]��k�>_��a�a��z+�?��|3�@}[������g�?���(���?����|����������������u�W?�|i�|s�.��i���� y�{@A���������������[����IJwAEVFEPEPEPEPEP��Y��R�1�!��g�t��xu��e\��r9����|l�(
QY�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@�s������y�z�?�,�����������O����?�J<��
(��"�(��(��(��(���Q@�(�_���Nt/���WI\����':�z�	����q��u�/��QErQEQEQEQER<�JZBp3@#�-EYuK�v��w������V5	�]*��fp	9'�5^�.[����(���B�(��(��(��(���������1QT�����t_�)�����o��%+��o��%QaEPEPEPEPEPW4�����
S��O�����������Mi>��J((��>�(��(��(��(��(��+�>;��������^�^y���H�������J��������EO�� QE���Q@Q@Q@Q@Q@z��_��o��_���5���b�����b�����������EW�$�Q@Q@Q@Q@Q@y��`N����Q+�}x��:���/��
���9�=r�~����w�������%��+�O4(��(��(��(��(��+����/�~����x�{?�O�e���?������(�((��(��(��(��(��+���������u�W[����������_��e�����EW�{�EPEPEPEPEPUuK�a����o6���}v�8�*�W�&��N���f�f�c9P��>��������m��x���Q��F�����`�+�8@0:s]=���<����m"���Ti��K;o��#'<p���a�A�]F���x����h���v��4��0.-�sa��d���xs�������A�������6����H7�#�z�)eH��P�/��
L�&����{!,�wW�o��� .;U_����w�*����6�c��E������w������������+��g������Y���9��<��=��c���Ov��3�]?��V3���y��|5g���_���o-����L�v���@R>b{�V��K���#�
�����h��N�����&>B��	�i?g�Df��������3�z�o�?�G��/�i?��|�Y����8�$�IS����?F�����O�>'���D�n�����3GW���H�g��V�����~-j[�����jb�Hdk~p��<���=��~��2���n�5���O]��O�;K)���� �?�4�����gc�>�`h��z�T���i���
���K�h��vf}�q�1�����������4������h���O)�Z���By����X�hE�����e��q%�w���F��G�#���y�����gnwIcmqw6?� g����0���$�����@�l�����S��|mVa�#�c�P���
>(�v�@��?x���{	fi)��t�x�G���~#N���Wmpv���Aq�7��8��?�/�B/�#|9�������Q�G���T�:�������O���<������q��F�v�������O����(�g��{�f��j�_h����:��I��8Ep��l�����-y�Xs+���_����_���w���~��Y��5�/F|���bF��4�X�!R��P��8�X��6q5��w��?��qH�P��U�����G>��X��%�1��|�2����XB�4c����������8������G�_���X| �e����w������<��\:���$~F�[sn����8�+�n
u_S�'����&p�-��U�u{�-�����;�By���:����!��������������o������w9�)�W��.��4���]� 99�Y5��g�g�w�I\[���09�*����
*��������|��?��1��1�z#�o#��u�w����B8�����j������B���wo��~O�\�n�F��dq���.c�6��yD9�`�����sU�U���
tU�5[�l����)@ <t����o'h�O'1>��n��|~��j������B������Gp�O�!qn�`��`�4
j�n��y>c(C������?9�*����a�_�J<��?��k��������?K�A3���y	�����JX����~]�n_�
������}�)w��?S@�T�3��F�SQh>��B�3���4y+�o8�~��P������@���iLJ��x���CA�SJ_w9�x�"P��s����h>��B�8�#�4y*�p�4�
	(������g?x�aR9���h�46|;�}�\g�������]%r�
���H�[jF8�x�u���q}�����?��q\�>6zt?�����en��������i���|�8������f���|��6�������hkZ$���S8���}�@h��c�7y�"����9��i��~�A�F���P�TgNe]��'��}��lv0�/��������@h��c
���3k�O�<{���Ba~�hm��[9���,�U��DV�0�������2H�����t�h����G�����6?,��,��Fu������@(��a#���q��X�^(�"�6�������s@h��cD��3j�G�<����a�f�3�F$a��@(��&�Lw�$lq��P�q,�a�y$�#c�l��'����1���|�i��x�����f� ��6x��Y��}����Cn�X���9�� tDo301+��Y����2H�����t�i~��<��~s����g=],��Fu������H��:���9�����,�U���������l�����#>f�$��6y��Y��Ic�y�"����4������������OE@�q,�a�y$�#c��)�����g��Ndc��@h��c
���3k�9����h�>H���Cn�Z���4f����h��a%a�y�{(duf��H���@(�>��<��~s����g�er3���9��_l������k����DG^^����kO���0��j��{�V90FNIo�5��f�����>b�A������k���U��Z+��_%�ys�;(QW�Y���I���$6m��=0?
!�o 2�����v�n���GA^�5o�_�4(QWc�n��kT�|�������G�n����y@1
���c?�>j��V��}�������w1'����`E�z�s�4�k7�\�;y;���[���`�4��*����
*��o�����Sa�GLc��~4E��C<����b����`~sV������B���wo���F,��F9=y#�h�n���������7c��g��������
t������s[�2y;�1�N@��I��M429�tM�qn�g���G5o�_���[�����������W9�2n~��K����~Q���t��
����~���������t$��0�c9�`|��������\������>�s���D��g���h�4EF!P�s���4y+�o8�~��P������s���4�%$�?�4j���.����h(bFs���P�}?���F!P���h�Wn��O�����w�X��|�<�9�`���kN�T�u�������/@N8#��-R�����sf<�dq_�ny�F����Q�7�����9�JO=y�hmB����y~Xq���3KB}��\���.�s2����lq
G�sJ�����+�������c���G�O4f�j\�����0���)�}1����K���T��JAl��q�1��G��r�h�qjw0��~V�����R
B�Y��y~^����8�{��G�V��&�q,�[�*94�5K�d����m�q
}�9�h�B��S�m��\?������r�9�����L`�c%��r5$�y[�dV��c0(�Gh�>�������<JX�x$����Ns�5Z������m���H"Q�g����A�SJ�p}�R��8�z�4w��)�% x�4�5$g<{�4w��)�Z���~��6Fs�4h��"P3����]��SF���W4�����
P��n9����2�T�V������j���z�R����;�*�YB��[~��bF�l��Y�.<��~s����g��NOEW������dc���[R7E�6�3���<P�*��a�|��m���cg?\�����Do3jg����h�^Kee/�*����6q�8�����l~Y�OEWK(RS"��s���s��D��5p�f`�V?���EV�Z1�mb	��g���
�|��I�l�y�5������?�[������ �Q[��
������/�1�>�1��B��p��C�C^�S�
�x�h��>C��6�t�"��^PP�B��?��Zt������y8����t^GL�s�5��5_�_����E_�Y���)���	%1n�s���h����g�s�l����g=1����Z�
����
*�z��W��N�q�6�G����b��dX��H��t��������������h�������W�T�0�l�����]Z�kE�'�]��94����_O���CB�~mj�w��0�'��m�s���������%�|��������
O5o�_���E_]j�n��|�P��1���
!�� �W���[sn�C��G�>j������B�[�/��Z��y/��Z�h�{�mZ�<�-�g0!<�����|���Y� �g�
�����X$������O���J�u�^A��l��"U$������Rpz�����	(�y+�o8�~��0�Pp:|��CBJ*3
�3�>��1)}���h�4E0D�����H!U�3�������T~J�+��F�%v����F��%�H����S��#��F������>��n��6M�L����~R����5��\[��a����w���8�\x��y]L�[��K�3W$�.e�9_��%q
���?���?i���	��JcLc����O:��S����\�<�/��Ln�(G��$Z��!�y_�b��P�~��=��{�(�+�\%��_�A������Q���`/�P�J9�h�=��\���.�s3F_��m�q
}�9��K��I�����c�tv�r�h��T�
8����?�Lc��Q�s�:yY��6aC��G��Z=�tU��+��hS����aRy������������R�������U�{?�O�e���?��#�R��4G���b����{�����2�>��ku� ���1�OA�^�kE3�]���SA�J�����^�q�SJq��1������s��@�u����~��D�8�#M���yK��p}�R��8��4w��)�% �=�)�I���h�=��S|�����h�$��f�A���{�@���:��J��k���b�DTm�239��0v��#�ol�����*�X��$g����%l���Y,a�n�3����z��X��k8�q1���6?,����'3.�����G>��OEVKc����i��x��[V&�y�X�z���@h������A���g?\������a%a�y�4Uw��I�~�H���j�QE_P6�Cn�2yO�8�
O`z~5b���qbc>[yFO�n}���
\|Z�����m�kY�����2�q��Bs��
��I�#8��A�'�k�v��\�=�����Ot��Hb\���8�go��Fn�#�s�����h^ ���{
���[�N�2�c|�F��6H���*�89�Y�u,�T���GJ(�������syq�x�X���S-��oi['�c�z��<'��[��]6���C^���|�g$m�������8g�dW>!���7�
S�ww���]���OF������
������d�C5������C����t4P�?�v�����&��x{W�1���\(��O�9���N�twr[�s����^j��i��O`9��z(��_��Ok0k�������/�x���������/
_]����j���q��8iGE�W����(����,|[sg���+W����l\,�U=�}��5��Z������_k����"���1G��Q��gER��}�^�������E|;�\d���]��� ;�����@5�|��+����'��z����(��(��(���~8������G�_���?��������������
��|��������Q_nsQ@Q@Q@Q@Q@���(���������_���Q���i��_EW�\[�#���XQE�`QEQEQEQEQEjxuA��m�",m�y�]-s^��)$������WK\u>6z�?���(����(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��~9��f�?������������ms�����������E�?��%UO��Q^�EPEPEPEP@�(�u�/���':�z�	���o�����������W�8��������
(��(��(��(��(��)#�zZC�s��|���}R��J��S�|���Vu�T�2�����5Z�.[������(�$(��(��(��(��*[o�����/�Km����E�b�N?���x�R����x�UQEQEQEQEQEsI��������:����^����j��k����#���t�������(��(��(��(��(��������g�����u������?�[����O(������JDT��Q_�PQEQEQEQEQEW�|���V��%���^S^���F-[����F-x����*�/��tQE~2HQEQEQEQEQEW�|fb4�-B��I�0�����k��3�����5�C����W;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Cy3�YM4P��l�}� {��5q�<{�|6v�n�&���Nc�����Y�e���>)����V������*�g�>!hz����|7��&YQn�
5���q��q�MV�W�I��E��`�hd�{g'��A�_�W���u��i��"���0��w���L��\w��EPEPEPEPEPEP-g�@w���'��k���$W����O����?����y?�\w���"���2�����(��(��(��(������{��?�|u��}����H�����>:���Z�����o�I�����QE��0QEQEQEQEQE������{O���*�u�/�%����@5�U~U���1_�_�%�Q_&EPEPEPEPEP�/S�����R���<{RI�X����������������6?�*�O�2�H�x
��W�����`z��L^)���a�l���d��[������G�%����������U��������r�z]���/�>����������6?�*��K|9�A�/�c����lA�Q��?*>�.���/�>����������6?�*��K|9�A�/�c����lA�Q��?*>�.���/�>����������6?�*��K|9�A�/�c����lA�Q��?*>�.���/�>����������6?�*��K|9�A�/�c����lA�Q��?*>�.���/�>����������6?�*��K|9�A�/�c����lA�Q��?*>�.���/�>��um?Sx[L����m�Y6���q��nO�����
-5��i�z����Q����V��77�\�S����:J*���?�>�����'��td�����\�S����:J(r}O�FO���(���?�>�����:�����}��j�W��?�o�Z����}��>�EV��EPEPEPEPEP�����k����DG^^����K6��l?�Du����S�"������*����(�H���(��(��(��(�uP:��������������W7���I���^��Bj�+��w��_�K�d�Q\�QEQEQEQE��=;�����>b�C
R�HC8��1���j���MR�K+;�����5Z�.[������(�$(��(��(��(��*[o�����/�Km����E�b�N?���x�R����x�UQEQEQEQEQEsI��������:����^����j��k����#���t�������(��(��(��(��(��������g�����u������?�[����O(������JDT��Q_�PQEQEQEQEQEW�|���V��%���^S^���F-[����F-x����*�/��tQE~2HQEQEQEQEQEW�|f
����~_5�1��c��u���_�:~���W�����;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@y���h���i��P����$�1�\~5��x��'�|e�lj�j	u�,����j�G�h�m�E�=���^6����������Y��$e#-�v�G�j��3�����Z����
��|����Ae��K��ue!�/>���T� �Bh��AEPEPEPEPEPEP-g�@w���'��k���$W����O����?����y?�\w���"���2�����(��(��(��(������{��?�|u��}����H�����>:���Z�����o�I�����QE��0QEQEQEQEQE������{O���*�u�/�%����@5�U~U���1_�_�%�Q_&EPEPEPEPEP��f
t�0��	�o>�3��yz��b��:`?�}�v�3�+���3�����G�QEpQ@Q@Q@Q@Q@��o�=5����z���O�����H������(�EPEPEPEPEPW���������exw�A#���u�^��G��l7�c�QElnQEQEQEQEQE|{���K6��l?�Du������k����DG^_��?�/����(��|l(����
(��
(��
(��
(��
QE���~�9�����M]%s?���_��?�&���	�����6HQE�EPEPEPEPHr:u�KA�n����U���3�8S��n=��X�(u+�<�3�������_-�*[���*DQEQEQEQE-��~C�]�����������1L��c�i)_�c�i*�
(��
(��
(��
(��
(��
����^����j�\��/m���5t�5�kK���Gh:Q@�E}�EPEPEPEPEP^y���H�������J�:�����Fu�����zW���������"*|��(���((��(��(��(��(��+������������)�V��#��^K���L��E�}���:(��$(��(��(��(��(��+�~3�6:Q'���b��N�1��S�:V�o�d��l������w�������&��+�O4(��(��(��(��(��+����/�~����x�{?�O�e���?������(�((��(��(��(��(��+���������u�W[����������_��e�����EW�{�EPEPEPEPEPEPEPEPEPEPEPEPEP-g�@w���'��k���$W����O����?����y?�\w���"���2�����(��(��(��(������{��?�|u��}����H�����>:���Z�����o�I�����QE��0QEQEQEQEQE������{O���*�u�/�%����@5�U~U���1_�_�%�Q_&EPEPEPEPEP��flib����:a���yz��bF���2
�d�|�3�W�W�f���8��(���9��(��(��(��(��(�U�7����t��=z]y������?��?�����AES��(��(��(��(��(�����G����V��;� ���:��v����}6�1�
(��7
(��
(��
(��
(��
(��>=���%�\���":�����Y���a��#�?���������yU>6QEzDQ@Q@Q@Q@����P��?���_��?�&�������Nt/���WI_�������_�$(�����(��(��(��(�##)i#�|��0}R���
3��>cU�������:�c;�����V�������0��*I
(��
(��
(��
(��
���?!�����ER���?����������4������4�E�Q@Q@Q@Q@Q@\��/m���5N�i?������5��H���(�t�������(��(��(��(��(�<���$gY�z��G�zy���#:�����=+��?�cC�q���>|�EW���QEQEQEQEQE���b�����b����|���V��%���^&�"����@{�Q_��QEQEQEQEQE�_�~��$��s����6���1a��`.T��9������W;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Q@Q@Q@
gU���qN�����L��/y�C}���j�������TN�v��V�������;��_��S���T�=�|��m;�������{���,sr������I���xC�������B(c��d3��S";��9���7�b�4���x���[�_�o��o�qs�i�����	3G������<~J��;\x3����/��\��.����8h�H�q���V,d�S@ ��Ex��j|V���=zk�<=����XE3F�N~����0F>���A��|d�������\�y�m���-fL�)<����zP���P0�9���]��#�d����sp����{�h%1���G8��>��S�O�o�h3\'�|F^�[	fi��t�N3�1���T��}�^�������E|;�\d���]��� ;�����@5�|?�e|;�\d���@�Q@Q@Q@Q@��#:��������������{��?�|u��~��_�/����������+��`��(��(��(��(��(��/�%����@5�U|��_�J<��?��k������b���6K
(��L�(��(��(��(��(�5����#L�6���z��?��^E^�����0��
�����`�u�U�����~G#�
(���(�4f�
(���3Fh����(�4P����Mc��C���.�����zk�����uE���(��(��)�QE�(��(����;� ���:�����$����k���(�M��}�(����(��(��(��(��(��~9��f�?������������ms�����������E�?��%UO��Q^�EPEPEPEP@�(�E}I���I���^��Bj�+��y�$�B��A��5t��N;�������B�(�@
(��
(��
(��
(��
C����iNF}��-G������{���;�V��+�]����-�'q����.[����(���B�3Fh����(�4PEPR���?�����[c���t_�)��q��7��������M�((��(��@QEQEQEsI��������:����^����kJ�4��H���(����B�(��(��(��(��(��������g�����u������?�[����O(������DT��Q_�PQEQEQEQEQEW�|���V��%���^S^���F-[�����Z�3��U�_������d���(��(��(��(��(�2����?K�6����8���zmy��p���
%|�u�c�������/���O��KE�5���Q�3@Q@f��E��
(��
����&��_��)^1�����&��_��)M
�QL���(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Q@Q@Q@]�B���R�����^�k�����������]uy���aq?������������u?
������=y'ETK����\z���
��
�S�����(kT&FbKH�$�^{��5��}��
�,��X]����,����A�����TjpFm���~&��� 3#�_@�dWW��	���D������3��w?������|o�Z��F������y����;M�h�����e���?��������|2��|D�����>�g�Gs>��
:oS�=9��/�N�lMk-�����m�����v�>��=���^E����]��p������7�=����I����Ks��,����� g�?��_|8������w��~O���?���T��c�;��E�/5]b�s��8iGE�W���9O
��~��0��;^���3�j������/�(�9���k��&�eV4�}�?�����|Yyi�%����X�-�")��<2����>�si��jmsP����nh��}~��q�up��:]g�@W���'��k�>|2�����D������/��w/0�U���Y����{����������q��F�v���ED�f����{���0�>W��v�����s��EVk$�����$`<�K%�2-��H����Q@8���;������8�[(Rc*��s���s��U�(�XAp�g�6������a���3kO�?�j��|cQa��Y����D���3 9
�x=���u��%�H�;�m��x���#�������$g^�v�|w_�pm*s��sE?}�Q8q
���:���f��9�	��$f���F���������7c��g��tW�{
W�*��S���6�w<q�!�llv�����9��Y��x�s��+�t�P�T(��z+����]���o>�.s�g�{�1��?#�o#��t0���n�q��P������v�.���7��o�a���m�#r}28�
jj�Q��E1yDs
��k�j�{
_����.I��Kf��b��1
�N@�>mf�v��0�'��m�s�����
(���}��������Gp�20B������h���k�a�B�t��1��
(���m���o���\�D	.�M��cU�'l>���1)`Nr?�5����(����������+�c�$��_���.����h*�Fy�h�������@ g��h�Wn�q���RQE�s23
����iL*q��1��>�,���1)}����@�Cn����EAvF!U�3����
G8?���,����+�o8�~����s�O���Qd��������`���d!�6F��=k���.>��l��m���s�����?���t����~�F�y|�`��;y5�.}�r�W3F��-�A�GN��K&�u,���-��W���?�S��y����]����F�O����?r����hMR�'�e1o�����c��G4��<�����a�<���f<�L�>���\%��S�A������Z(�}��]�O�\Ij-�����J��,����3!��lq
}�9��E��sK�q�K���f1o�����4
R�\����O�S�cN�9��9����u�H�-��aB?�?
H�+��h����'0�<����E����z��	^kId�!x�a@��:�5��5�����|��Mc��C���.����+n0D�3�^MR��8�z�}s>��.�JTp=�%8�x�4�(�}��]�����s�4�6�s��QG3g�`�Fq�}�%
G8>��E��sK��)v��g=M% �:ri�Q���4��1� ��f�-w��?Zus0��s������g������1�J�KW�������T�;� ���:��r����}&Z1�+&/�m-��cg?\��4I�6�$~����j���ZKd	���F$a���9��y���x �#��qS�@-�K9�o�I<�����lv0��n��v�������@������3������y"/�m
��cg?\��(��@��7��������a���~T1#��X�� �_h��}�������R%�1����0 �F=}�V(��>5^Mi�S��-Z�tj�&��#-���s��5����P��1��?�~9��f�?��������z��Ry}��������v_Mf�.d�L>d�������D:��S���]�[���#���W��z/������z��Vf�^S`By��c4���Ib-��b
����{z�:)�
W�*��u�e������a���]�"�=p9�Y5��.b��;�.-�z�c��R���W��.��Y�f�0��6�t�>���DZ��3�*w�A|������QG��������:���Of�1f�1���SF�t,~�^N���Sv?��Z�E������e�u{���;�1�N@��I��M429�tM�qn�g���T(EW�����g��&����$�zK`������J�0�P9��h�?���I���^��Bj�+��lR�UI}�~l�fFaV�s����)�Kn9����ErY3"P������
�q�F>���E�]��T)����J���3��jJ(�fFaR9���S�	��}Y3�.����h(l����O�� �#����4�!8���IHs�={Qd����P��Z�h�6o5���@c���j�K�ZB�,���0���#��� ���8������7Uz��'}�2S��-&�q��L~Y�s��^q�P�{1l�?,8�A����R�}���r������C#`��Pr>��W�.d�9X��<��(>���tQ�.��.���;�����Wg���>���Dz��SI*�JAl��q�1��T���]��]�qjw0��f,;9�'�8��B�l��~^����8��5Z�9�p��r�����,Nc���!Px���I�]K$n�,����������G4��<���rnrb��T~�1���T������Pb�#���S���
��m�������Q�.���s���K�������s�������x�Us>�sK��-Cn�?ZA��<�u4�(�}��]��)���<����g=M>�9�p��q�% p=�)�I���E��sK��-wn�?Zj#9��us0��q�%�<��<��G8��}s>��.�<��8����@�j�����1�W��O����������:�4�){H�����J��c0{f�Y�.<��~s����g8�E{������6b�������i���w�|g21�s�Y��+}�'����������sCX@��6���bF}��Y��+�c����@0�~4��#q��������8���
�e
Jd]��s��}�H�F���a�dc�x�4Paa��f� ��6<�l 0�G���#����5f���H���a��q��\��	u����!~c��2�����������g�����e)<����S�g����5��&/((^!@��c?�:mb��cY8����y3����+��aK~U�]�~Mf�[��s�I)�t�P�G����>��|��?��1���1�z�EW�kr��.���7�\K2w����B8�`~��v�"�a��Y�@�����?
�EW������qu[��6�b�J��-���3����]Mh��b��n1�N@�R����{���u�e�����&����z��A��?CCkW�t�a�#R��t�5B�_W���_p]��Y�[��1�)�GL`{c�u��$��0�V���C��G�P����zr/�.��j�p���|��������3^��2�k�gP�]�\6K�lj��`�2z��"�V��#��^K���\��5��j*�t�Awc�J����4�
@�_���W��D�1�J���3��i*T��1�(��9��[�x�����s�4�(���%
�g?�A
�q�F�jJ(�fG��R����h�Wh^p~��(��9��H����S��#��O�� �c<�����My���i�cFpD�py�z��k�+�~3���[��6�03���;��_�S:�|��)5K�&�V1n������c����;�����g���>���T�����s���r�j�1�$�b�&7f#�A���-N��_�b��P�~���E����Y]B�-
�����%�>��,�����1�j�A��8�U��g�9���.�u3F\���r���sCj�-p��B���5N�9��9���5K�p���P��)�}1�#�.��GCdm��P�����T���]��]�Q�W@���cg9�I��$f�������m���s�����E��sK�n]J�h��� �B����9�`�]<���I���[���6�a�Z�J����&��_��)MI�������q�3�����sO��3�W4��1)�s��SJcR�����E��sK�����~��%�y�4�(�}��]�yK��p}�R��8��}s>��.�J@<t�������f�E��sK��-w��?Zj	#?��QG3iw"P���S�Z�&����df8r9�s5��{�@���:����k���I��������#;��q���>����6���H����Q^��5�M8���?��qB�����~�I9����8���
��A}�g��Ndc��B�@�4c���������EV6D_��G�9�����DEo301#��Y��+��2H���(bF=�V(��
(��
(��
(��
(��
(��
(��
(��
(��
(��)k?����y?�\w���"���2����Y����{����������q��F�w�QEQEQEQEp??�������������O��Fu��`�����u������LNO���(��9��(��(��(��(��(������x?�����U����(����������-����
��,(���0
(��
(��
(��
(��
(��<��1Q�������8���b���w�3cH�F	��}0��?�y|�a���_������(��(��(��(��(��(���_�����H�����|��Mc��C���.��QE0
(��
(��
(��
(��
(��:���$����k+���	����Z�h
>��a��@��+cp��(��(��(��(��(����Y���a��#�?�@���%�\���":����)��}�G�IG�S�aEW�@QEQEQEQEP:�(E}K���I���^��Bj�+��y�$�B��A��5t��N;�������B�(�@
(��
(��
(��
(��
C�N��i#(�A��;��a�Tc�j�Vu/�]�R��rT�1��W��vyr��QRHQEQEQEQET�����t_�**���?!�����0>�������������,(��(��(��(��(��*���!{o����usI�������������/�G���E���Q@Q@Q@Q@Q@y���#:�����=+���>;��������^�Q�#�������3�
(��~<���(��(��(��(��(�V��#��^K�����[�/��Z��y/��Z�3��U�_������d���(��(��(��(��(�2��TXi@���!��b�6���3cO��i ���L���W;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���������;���_��?�k�c���������
q�����w�����^�;�(��
(��
(��
(��8��Fu��`�����u�'��#:���������k���_?���&''�AEW���EPEPEPEPEP�_�J<��?��k��������x?�����U�W����~l�QE|�Q@Q@Q@Q@Q@k����F��6���O���^E^���7�F��6���_������W�f���8��(���9��(��(��(��(��(�U�7����t��=z]y������?��?�����AES��(��(��(��(��(�����G����V��;� ���:��v����}6�1�
(��7
(��
(��
(��
(��
(��>=���%�\���":�����Y���a��#�?���������yU>6QEzDQ@Q@Q@Q@����P��?���_��?�&�������Nt/���WI_�������_�$(�����(��(��(��(�8�=;�����@1j%��vd_�}��w�Vu�T�����;�V�������0��*I
(��
(��
(��
(��
���?!�����ER���?����������4������4�E�Q@Q@Q@Q@Q@\��/m���5N�i?������5��H���(�t�������(��(��(��(��(�<���$gY�z��G�zy���#:�����=+��?�cC�q���>|�EW���QEQEQEQEQE���b�����b����|���V��%���^&�"����@{�Q_��QEQEQEQEQE�_�g�`��}�=?��^�^e��7�~�A|��1��c��u���w�������%��+�O4(��(��(��(��(��+����/�~����x�{?�O�e���?������(�((��(��(��(��(��+���������u�W[����������_��e�����EW�{�EPEPEPEPEPEPEPEPEPEPEPEPEP-g�@w���'��k���$W����O����?����y?�\w���"���2�����(��(��(��(������{��?�|u��}����H�����>:���Z�����o�I�����QE��0QEQEQEQEQE������{O���*�u�/�%����@5�U~U���1_�_�%�Q_&EPEPEPEPEP��fRt�5�@
����?Z�*�����I�	?0��3��?�W���f���8��(���9��(��(��(��(��(�U�7����t��=z]y������?��?�����AES��(��(��(��(��(�����G����V��;� ���:��v����}6�1�
(��7
(��
(��
(��
(��
(��>=���%�\���":�����Y���a��#�?���������yU>6QEzDQ@Q@Q@Q@����P��?���_��?�&�������Nt/���WI_�������_�$(�����(��(��(��(�'=}�iGN���-EJj�j�\����<�j�~�����r�g
s����W�������
(�����(��(��(��(�m�������TU-��~C�]��`}8����IJ����ITXQEQEQEQEQEU�'�B����CT����!{o�������SZ_���;A��J+��
(��
(��
(��
(��
(��
�����Fu�����zW���|w��3���o����<��F4?��)S�g�QE~�yAEPEPEPEPEP^���F-[����F-yMz��_��o��_���g��,���4��E��!EPEPEPEPEP^e��I���@�#�����M�1���lt�'���b��<w�����u>y5Q_*y�EPEPEPEPEP^����D���O�+�+����/�~����44w4QEQAEPEPEPEPEP]o��������J���?�o�]�?���C/�7���:(��s�
(��
(��
(��
(��
(��
(��
(��
(��
)���}��N)�QH����SK@���,@�����QHX(���FA�����I���0a� �Q@���������;���_��?�k�c���������
q�����w�����^�;�(��
(��
(��
(��8��Fu��`�����u�'��#:���������k���_?���&''�AEW���EPEPEPEPEP�_�J<��?��k��������x?�����U�W����~l�QE|�Q@Q@Q@Q@Q@i��o�N����������^G^�����t����|t����J�*�������#���Q\EPEPEPEPEP����Mc��C���.�����zk�����uE���)�QEQEQEQEQE��w�A#���u�Y^��H�}��j��C�Q�>�
��Q[�Q@Q@Q@Q@Q@�s������y�z�?�,�����������O����?�J<��
(��"�(��(��(��(���Q@�(�_���Nt/���WI\����':�z�	����q��u�/��QErQEQEQEQEPs�={QHFF(���R�0����n:cq�W���.�)P�9���_-���vQEI!EPEPEPEPR���?�����[o�����/���q��7������7�������(��(��(��(��(��O����������'�B����CWO�^����Tv���W�PQEQEQEQEQE������?�[����C�<���$gY�z��G�zyG��h�?�R"����(�������(��(��(��(��(��[�/��Z��y/��Z���o���1j�����1k����YW�~hs��+��B�(��(��(��(��(����9O��[���&���Z���/�����92�0������w�������%��+�O4(��(��(��(��(��+����/�~����x�{?�O�e���?������(�((��(��(��(��(��+���������u�W[����������_��e�����EW�{�EPEPEPEPEPEPEPEP�|k�q�����'��;�BmB2>�t^8��*������X�#g;Uc��b����?�j�����
������H6���x���/e�v=O��57�Oy,u��x���H����5�����C���<y5�:Q��N������n�$c�������������	�?y~pG��^?o���gm�����������$�����D�����j�){��M�m?N�Y�8���������V�������O}���H��u0\H]��F�Fs���/�����ti�[�-~w�PO������JZ��]/A�p��Y�!O������+x��^�y<=�����G3F�N~����0F>���A��|d����s;�\�y�����-fL�)<����zT������K��nS[�S����G�#��)?�?�o��-������|���L�Z�]�o�.|+�\�����{�h%1���G8��>��j|)����=Y�������a$�"C8�����r>����R4��3��k��Au�@�����/�(�9���k��&�eV4�}�?��H��}�^�������,������5����}�^�����G�/k�4MKP]@��B���N�%�F*�Q��(�(����ED�M�9�����i~���a�n������s@h��a����S8���}��,�0�T��1#�@(�
�Mq��������P�P��U��9����8�	���aa���������, X�0kO�?�h����$g^�v�|w_a�cQa��Y�����e�w�����<��\:���$f�����}�~����/�����[t��8q�E
*�:���f-��9�	��$f���F��d������7c��g������}�>�����N��6�w<q�!�llv���t�id��%�)���"Jb�������[�W��4(QW���>�.s�g�{�1��?#�o#��u0���~`B8�����j������B���yo��F��v�7'�#�����u��S�AS�P��X��G5_�_���E\�U���m���(b����|����Ha�O�6���|~��j������B������Gp�20Uqn�`��`�4
j�n��|�P��1���~4sV�U���
�����x?�����U������~"��&�g}����A_@��'9�x���V������������L��~�s�4�I#<��k�5����*���1��]�y�s�����Tf*8>cA�[�x��hIE0���s���D����h�4EF!U�3����
W������T~J���3��h0����4js����i�
�d�������W�W��]���I�0l���>�
��1����W��h\}��������R�\f�k0������l���.�s4i���b:r,�����#��FI\B�~#�5���Z=�y�5p��F�O�����S�c&�t�L�-�0�{c�t-�<����u����x��0���G��.����� �bRy��h�=��l����������~XD�������.�1�_�`��Ps�����������3WT�{��c���P}��@�.�����]��)�}1�=��=�y��G�]E$�!�t�-�P�C~�jW1B���kNaRy��{��{���k�=5����z������:���B����?\u�k��j	#?����v����y����]���SF���E0��@���bS���sF���E4���s��5
���h�4E0D�8�>���#�sF���E3�]���SA�H����4w��)�5$���������F����w�A#���u�X����;���p~s�}+I, DuP�q���^+���8�I��}4Uo�@!1a��������hk$��������>����4Ug��@�����bF~|���'�L���0{g=�����}�����}��lv0G�`�v������,�U������3������y",>����6s��Y��=�#�H���J�0���*��O��,QP}��y�}�������R%�1����0 �F=}�@"�s������y�z?����>/�p�Se�B���X��9$e��u���[�a�J�t�>���_�eN��}E|����.v�e
*�kW�s$�a�$6m��=���Q�y��a��m�!��#���G������(����QY����������iU���Z1���P6O��i����~�w����:*������Ja���]�"�=p9�Y5��.b��>dY��z�c���[�W��4(QW��x.��0��6�&1��?"��!�YP��b����`~sV������B���wo��k�f�1���SF�t,~�^N���Sv?��Z9���_���J:��.�w=�C!�bF @x��4�5����G0���.-���9�h�������K|<��s�������J�<M���I~����o�3����aR�s���������[�4�6N��Tflg<���������k�P�}�����4Us��1�������*�pz���%v����4jQQ�T�8>cJbV �����P�}�)w��?S@�Cdg?�5
�G<S*<���� B�v�����CC�]E�j�l����J�8;�[5�y�]���HL>kH���H�����-R����26��(y�G�����.[�S�j=F�;Sn�?-�������f��.�[1��"\������t��3V���fT�\B������u$���-��n!@9��
�{��y�5s�R�\���Wg���>���Dz��SI*�JAl��q�1��G��r�h�qjw0#�f<;9�'�Q� �.�����m���s��3G��r�jMJ�X'1�\c(<t����.��7scm��Ps����G��r�j[c���t_�*s�]5��L[�J���0}����S��J{�tV��c��G�;G�����o��6��)b9��94��Ns�5Z������m���H"Q�g����A�SJ�y�4yK�o8�z�4w��)�% x�4�58�x�4h�q�S|�����h�l���h�4E0D�g�sG��q�>����>�i?������)\s���zd	&�l��������:�4�oiTwUv����}�����p������6?,����������1@�`A��x?�"�@��jk�9����(�[�0y>Vf���9��45�#����u��4f��%�2���H�����8M��C���6?,��'����))�C�9�db9��"XA�P�q���^(�XX@!h�}�A?�l�y��@aX�}�I�l�y�5�����k?�[�����{$TV�F$a���p_c[����y���1���C�C^�S�
�x�h��>C��6�t�"�������������6�yp���F��l93�����<�o������Q�����Kq�a�	%1n�s� ������}�0��<��	�g=1���j��W��4(QW��o"��t0����L0?
Hu��u�c0�F.��F���G����}�@��E\]V�,M�����y�`��������SZ-������@����3N�o������~!�J��6�y;��s�]��9����hmj�����|�����1���\���}��B�}u���{�a�B��t��0(�Z��I^3em��t<�dq�Q�[�W���4(W�|���V��!����-^�V�C�����z�Fk�>^Mq��JS���vm�T��Q�'�z�s�S�6����|��ic�(��I�y�h�T)8=~c_��N��T~J���3��h0�Pp:|��CBJ*3
�3�>��1)}���h�4E0D�����H!Q�g���5
	(���
W���J���F�CBJ*3
���4�%-����h�4^e�����`.T���N?<���+�]������|a/��b��=��rr�_�?�q�����u3�nFy&h�\�T��h�s��+�P}F9�h���7"|�����1��1���{��h�)�����.��I���Ln�(G���)"��a#1�F,��'�8��G�R�����Y�e1�D��s��3D�����n�?-@(<t��@�J��5r]R�f��b�m�q
�
�]=�LL[���1��0h�B��S���T�[�����?�Lc��Q�u�:�#nl���l�?
=��=�tU��+��hP���������3Hu�g�bc����T��_�������V�{?�O�e���?��#�R��4G1�Bb��5�����R�>��ku� ���1�O���v�S<����g=M%*8���{���L1)�s��SJcR�����A�SDjw9���g����{���yJ�p}�R��8��4w��)�% �:riLjNN3F�����������@�A$g�sF����|=� t�}��q�%��}�u:�Rh���Fc�#��f���w�-�����EVk$������<���Kd��?�6�H����c�,QP5�-8���?��qB�����>�I9����8�	����A}��u�s#?:�����������,�U��h9�l����	��$a�y�4Uw��I�>�H���j�QEQEQEQEy��>�������7�E���rXmF�.r�A�;WI���~���/�z�������C������]U�j�
�������j���9�]=�d�����{��<��/�Z;X�BWi�3\\������gn�����_��_>���:��uY������GM�x'�>������
��6��^�_I��_�>��os�9���j(��W��O�P��~�}�kP������i#����
����<'ww�������.u�
+���*��z}+���9?|?��e������V�b��l_l������x>�Z���m<;�O�j����M�o���?��(�������������
q�����w�����^�g�@w���'��k���$W����O���QEQEQEQE��p��3������;��>8������G�_��\�"�����181?
(����
(��
(��
(��
(��
(��;����Q���i��_EW�������^����������F+�+�d���+��(��(��(��(��(���_���di�(�l�u�x�������~3.t�1�x�~=r��o����c0�x/���|aEW�QEQEQEQEQEz������?��?�����>
�����]!�O^�TZ
(��Q@Q@Q@Q@Q@_��?�o�Z�����������{�?�C������QE��QEQEQEQEQE���?�,����������|s������y�~�������#���������+� (��(��(��(��(E��>��y�$�B��A��5t���<��s�������J�'��W�R��!EW Q@Q@Q@Q@!��"������Q���v��w������Vu��](b�fq�����_-����aET�QEQEQEQE-��~C�]�����������1L��c�i)_�c�i*�
(��
(��
(��
(��
(��
����^����j�\��/m���5t�5�kK���Gh:Q@�E}�EPEPEPEPEP^y���H�������J�:�����Fu�����zW���������"*|��(���((��(��(��(��(��+������������)�V��#��^K���L��E�}���:(��$(��(��(��(��(��+��3����l(��1������M�2���������3�����;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���������;���_��?�k�c���������
q�����w�����^�;�(��
(��
(��
(��8��Fu��`�����u�'��#:���������k���_?���&''�AEW���EPEPEPEPEP�_�J<��?��k��������x?�����U�W����~l�QE|�Q@Q@Q@Q@Q@i��/�N�O���o��3�+���>3N������E�b���c0�x/���|aEW�QEQEQEQEQE�����X����'�K�4�7����t��=z]Qh(��`QEQEQEQEQEu~��H�}��j�W��?�o�Z����}��>�EV��EPEPEPEPEP�����k����DG^^����K6��l?�Du����S�"������*����(�H���(��(��(��(�uP:��������������W7���I���^��Bj�+��w��_�K�d�Q\�QEQEQEQE����C���j��� ���8��������V/�R�h���F1��U�����KvQEH��(��(��(��(���������1QT�����t_�)�����o��%+��o��%QaEPEPEPEPEPW4�����
S��O�����������Mi>��J((��>�(��(��(��(��(��+�>;��������^�^y���H�������J��������EO�� QE���Q@Q@Q@Q@Q@z��_��o��_���5���b�����b�����������EW�$�Q@Q@Q@Q@Q@y��p�a����Y6�03�+���~3�:P#�2�T��?�W;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���������;���_��?�k�c���������
q�����w�����^�;�(��
(��
(��
(��8��Fu��`�����u�'��#:���������k���_?���&''�AEW���EPEPEPEPEP�_�J<��?��k��������x?�����U�W����~l�QE|�Q@Q@Q@Q@Q@k����4��!����s��*������4��n'�����U�U�����~G#�
(��p��(��(��(��(��(�~
�����]!�O^�^i�o�=5����z����QE�(��(��(��(��(����;� ���:�����$����k���(�M��}�(����(��(��(��(��(��~9��f�?������������ms�����������E�?��%UO��Q^�EPEPEPEP@�(�u�/���':�z�	���o�����������W�8��������
(��(��(��(��(��)��JZC��z��Qb���*Zw%[���Z��%��vdP�f}���5Z�.[������(�$(��(��(��(��*[o�����/�Km����E�b�N?���x�R����x�UQEQEQEQEQEsI��������:����^����j��k����#���t�������(��(��(��(��(��������g�����u������?�[����O(������JDT��Q_�PQEQEQEQEQEW�|���V��%���^S^���F-[����F-x����*�/��tQE~2HQEQEQEQEQEW�|fli�Z�$\��������/������T��9�p1�*������gS�g��E���QEQEQEQEQE��	?�M���d��R�b���'��������
SCGsEUQEQEQEQEQE��{�@���:�����������u���/��2��|�����+�=���(��(��(��(��(��(��(��(��(��(��(��(��(��� ;�����@5�|��+����'��z�u��������;���_��?�k�}EPEPEPEP���H�����>:�������$g^�v�|w_�pW����7������h(���s�(��(��(��(��(������G������
}_:�����{O���*�*����������(����(��(��(��(��(�~3n���0F��6�����k���~3t�0��.#�+�1���W�f���8��(���9��(��(��(��(��(�U�7����t��=z]y������?��?�����AES��(��(��(��(��(�����G����V��;� ���:��v����}6�1�
(��7
(��
(��
(��
(��
(��>=���%�\���":�����Y���a��#�?���������yU>6QEzDQ@Q@Q@Q@����P��?���_��?�&�������Nt/���WI_�������_�$(�����(��(��(��(�8�=;���|��������{� pN�U����5K�f.D���<�j��n�._
(�����(��(��(��(�m�������TU-��~C�]��`}8����IJ����ITXQEQEQEQEQEU�'�B����CT����!{o�������SZ_���;A��J+��
(��
(��
(��
(��
(��
�����Fu�����zW���|w��3���o����<��F4?��)S�g�QE~�yAEPEPEPEPEP^���F-[����F-yMz��_��o��_���g��,���4��E��!EPEPEPEPEP^e��w�~��6���w'��6���2���l�%pG�@�X�k���/���O��KEW��hQEQEQEQEQEW��$��6_����AJ������&��_��)M
�QTPQEQEQEQEQEW[��������������O����f��#������#N�(�\�B�(��(��(��(��(��(��(��(��(�k�xf��Kk�i6���d�[����[ ��3[����}R��2���eOc��W��j���$�����+��Ug/b�T/������2xB�G{�.�,��5�a2�� ��(gP�t�!�mF��\�Z�u�1��Fj��@�[J�D�)$lXz�:��|>����'Z���Yg��,,`�����T��G?Z��xW���i��F��MR���+n��u\����z���a�A�����Q�%�����T������qcq��e�@��B8�"���c�'����?@����r4�>��(�����$���R��P�>����/��_��r,��.�'%Gl����@�{g��5��u��Y���<RX�6Z����o-�!'[yVE'�	�
�[|F���������6��R�i�.����B��K��q�������-/��%���\E���8^������=SY����{����������q��F�v:�������O��
�OP������k�#u���*�?z��@�E u#!�Z7.3���-�����,���-��q��L��g���I�OB?:7.3���p_?�������������O��w��t/$�8�?��:��F���:�_�pS_�����bpb~46�p���V#�lm���z�����E8��d����V���@6�w��������rH
�F(��S�ns�c�lb���z���E8��S�\Pcq������E;�|���zQ���ll�b��;����Q���i��_EW����xr�������#"�+���/�����E&E�b�L�2(h���"��L�2(h���"��L�2(��3t�0����3�+���U��B�'L�/���:
�?���km���\W�����8��J)J���~l`pU����J)v6q����66q������+����k������k�������>��R�l�kg�ll�kg��R�c�[�j�����T�7����t��=z]y���,�|�fHq������h(�#���QFh�QEQ�3@f��Z(�4f�:���$����k'��?��#��z���������(�K��}��r�9���q��}kcqh������q���-��q��L��OB?:Z)7.3����q��}hh���������~t�Rn�F}3F�z�������k����DG^^���K|f���A0`�������ll�b�~�Z������(��|lm�-�#kdu�F�8V8��zWD
����)��(��w8��@6�q��2�3��<����b��E;�}�����P#rH�x���N�
��lm���z���@�)�7%X�[�F<qE�Q�<��s�������J�����C�-s��WG�_�c�����_�$Z)2(��@�L�2(h���"��L�2(h���"��������j��PUMN�P�Vg�9�z�Vo�N��S��}��q�W����+�����v%�X��8��[�q��D�E)VU��F�[?JJ)v�q����61$l��PQJ�E'��km���\PQK���S��c*������������1Q�l�kg�-���!["E����M����x�R�cs��:L�QFh����(�4f�
(���3Fh��O�����������$
^�'�������SJ_���;Q��M�q����q��L��P-���#��r�9�-����������������~tn�F}3@E&�N���OB?:Z)7.3���������������g�����[�u#��<��s�oY�����n����1��8�h��>A������q��n1�a��W��G�6�q���d���}����q�.�m��@V��1@��p�q��.�m���v��\PQ�����(��S�n1�a��Q���ll�����N��v66}1@��p�q��.�mz��_��o��_���b7#![��U���H %�#���L�������@{��dW�D�E&E�RdQ�@E&E�RdQ�@E&E��_�M�����Y��3��zfEy��m��J�/��������������������R�`@*�=8�kg[>���<�(����+dv�X�V?�%�[�q��6�*q��J)J0��?
66q������cg[>��#��{PQK���S��v�z�����'��������
W��a�H�+����
�0 ��N����wQ�3TPQFh�Q�3@dz��(�4f�
(�����?�o�\�k�����H�����0�����_�jQI�q�~tQ����s��M�8���h�3���hh���������h����F>�oQ�~t�RnQ�����
(��
(��
(��
(��
(�� �;m����Ig1�����u�&�*Z���2���E�M�-r��[����~��u�W�S��5���b�4��i�H��c�����k����tOi����)k6�%�V��1����J�}��|7rv_i�����[���Ie��i�b�����>��SH��+>���l�+��7�x�V�����;<����Z	$_F#����GF�v�l4_"72Wf.�����$����H���A�;.�S�|(�^)=q���jl5��kF�����iR�vW�Y���~u�x�������/��-�dw�3�3����}j����x.�h�Kw\�����2M1�Y�'�P�&]�����vv>�����B�0_\s�_8�>7�H�"I�mB���I��_��]��|�x�[���ujI�����4Y������?���d��K�ion����c4��=��(kY����{I��������(��
[��u�����y��D�O�^����������
q�����w�����^�;h�-"����5F��/�m|�+���y��o�������X��2���W�C��HT�lTar�OEBl��9�C.A�����&2��r�y��ME@�6��	o�l/QH,mV&�[��J����(��������-��$�B���T��~��&-gR�I$��ty[t��X���������{��?�|u��~�����/�4S�����:.E�j[��L�>w"����HuK�g�Cw1���+y��LUJ+���;��w���.K����5����eVrB���Y5�JY��K��H�1���s�J�=�/�_p]��u/���O���y�v�L�P������{:�.<��:d�*(���}�v]�X�m��"�x��.�\��z��LMR�;3h�s-����=F*�{_����-����j-����P��*1��t���;F�_N�'�i	�����QG�������k:��G;�Ne��d9Pz�h���-p/�2�i<��l�*){
_���g��������q<�,�L�l�`��k���O�+�����Q���i��_EW��W�0I/��6Klg���m�q@��A=x��_+`�!�(�h�c��b�9�)�R�����Qp:t���QN8S���.�����w�( m�@>���L.�c�(���c
@E��1O���]��c��b��1A�2(��t��E�]�s�z�[M'O�<M,��P�*������w�_�}�_#|��.=1^�����i�.A�l�L/�O�^E_5�6��p�����&�{,j��L��Vr@#�+�w�I�]���1s�>�V���}�9�����|n��o4.���q��S�I�d��d�����5V�9�p��r�z��;�����b����OsM[���6�s*�A0�.^*�s>��.���.���s+B2�t��&�}1C-��cm����}EU��g�9��������5��H���W=q@�����]����_y����Q���4���S��I;���9vr���=F�(�8��DbK(r'�V��g�9������5����:����~���4!@'������zk�����t��\�����u��M��1��u��q�K��-
�P`t���Pq��us>��.�|�-����(�m�F}qN����}���g
z�G��H0z�S����iw���l��4 ������(�}��]������j<���h��N����}�����f��Ro,��9#5��6���o��0��xw�A#���u�^�����L?�c�W6�#��a;���>�����X��2�IU+��OEln@�6�[�������gn�	Z��1^x���V��f3,($$��y�KX�ypF������=���m����B�qG�m|������x��OEWkWDG���(+�����ug�6eW�:T�P?c����%<��~�sH�V��t�5f$/'=jz(�?�w��5Hm��(����#&�#��c��-r��s2��'�r�k����%�\���":����*�M����#��]F��u5�E.t��e��$9`:d����d0���V��d#q�5J��}�'�W�E�m5[��
�w�,1�;H=x���K�,������".v�:qU(��i�����{��rmcQ���g�D��t��#���j2\G;������d9\���TQ�)/����.�cQF�^���d������=gR�ie��t�b��X���TR���_p]�!��#x���4rY�\�I�A�_�?�����o�����*���+��}�v\�V�&�a��w�*������5�Ji��[���m��HIS�*����R�U���7����}�Ip|���3�rX�nMtLe@(�+��y�$�B��A��5t��n5%����K�d��0�q�S���JH[qPO�)�W%�]� m�F�\P"�s�Q���>�vAv0CR.Q�<������q�}���a�2(��t����QI8��E�]���v��w�( l�P}qO����`�0��u�0���zb��<�x������W����/'��h��a��p���S����w2s�r7SI��6�v�����=��V��rw��%)_r�j���#��b9�89��#_����k�L#-��t���K���.��5+��	n�qP�N�:C�w�L����$y���W=qUh���iw-i�}���\�i]�����4&�}�I���]���:f��G3���Y�R��b���f
�d���_��_��D����q���E��sK�fMF�XV).�h�R�)d���x�K���;��'i�V�9�p��r�����1����U_y���-u+�����o2GEv�r�#�Tj[o�����/�s>��.��O ��$:R��J�}qO�������e�>�|���h��4����S����3�4F�`���-6�`�\b�E��sK��A���hq���(�}��]��i�v��������Ea��������)6�`��)�Q���4�����6���ZdI�[,��.�`�cU����!{o������:�4�'�#�Tu�gl�	^��a��1��Y����b\�~�sS�W�}9Y[FX�jX�/Ph[T��m��yB�jz(���>W�����n�3�CX��h�oT��W��������e2A)+�
Sgng��(e�w��55�[$�D�5s��^Nz�%��j�-�P��b��X���1oF ���4S�m�(�!v�
OEW{Y��`�j���p_cX>�70�I���El���"��������g�����e)<����S�g�m�_5���cn_(�������FuE��ylHN�:T�����/�_q���d�ug�i/�ia����+����cQ�W�~�?��g���z�>�J�=�-�W�e���F)��;��Yq�8����4������������+�����:(���_p]��T�K3h�sr
�A���I���j���L��6��J�t���QG��{���u�����;F�_N�oB����Y���'k���
��T��E/aI}�� �.���-�\-��gP��C�A�"�u(G��ti[s������*(���}�.�����6�W�$M���s��z��ww����$P�/���	���Myz��_��o��_���������Q]:y�6�{��p�3��1�@E��O�����*=�v.3�b�e@(��)�QdcQ�2�q����Tn��>�,��`�n
����c8E����EAv3��)]���b�&=�v.lS���.�c �@�q���E'��(���Rn��n��y��)>�a��r���e}T���^�^e�����`.A��>��O�\x��y]L�7��/}N�Ic�K���$�9\���w�h��o4.���q���E|�3�y���Z]N�&yR�e�Lo`�-���=J��+��H��k�����Z(�}��]�v���neA�7���C��[�$���\����W��f��Z�S����w3��rrv�QCjw�2L�s�\����G3���Z����s	Yv��rG��w�;�ws#Hw9~c�j�s>��.�����h���cl�@�=:Cvm~�ne�1������^�9�p��r������-���AP�Ht5�_g���2�w#L�lu��'S��J����&��_��)MI�������q�c���
��)�>�sK��e�N(�����>��QG3g�h�d(��(�����E�9�q�Z#`��1G��q�c��(�}��]����
N:PcBrT��QG3����-7n�7z�#@r��QE�s>�Dh@���6�I��I2��GS���W[������������;�
�������6�����L�R���6�m� ��.W����X�Z���41�y^h[;t����$$��y��ME@�6�����]�kaz�JAcj��koF �����(cjbx���x��m]U^�6TPW��Pem$����	^F:T�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���������;���_��?�k�c���������
q�����w�����^�;�(��
(��
(��
(��8��Fu��`�����u�'��#:���������k���_?���&''�AEW���EPEPEPEPEP�_�J<��?��k��������x?�����U�W����~l�QE|�Q@Q@Q@Q@Q@k��w�F��m�Cn���Z�*�����di�(�l�u�x���y|�a���_������(��(��(��(��(��(���_�����H�����|��Mc��C���.��QE0
(��
(��
(��
(��
(��:���$����k+���	����Z�h
>��a��@��+cp��(��(��(��(��(����Y���a��#�?�@���%�\���":����)��}�G�IG�S�aEW�@QEQEQEQEP:�(E}K���I���^��Bj�+��y�$�B��A��5t��N;�������B�(�@
(��
(��
(��
(��
C�s��-!��"�G�Z���K�3�����w�Vu��]�m�'p[��y���r��\�&QEI!EPEPEPEPR���?�����[o�����/���q��7������7�������(��(��(��(��(��O����������'�B����CWO�^����Tv���W�PQEQEQEQEQE������?�[����C�<���$gY�z��G�zyG��h�?�R"����(�������(��(��(��(��(��[�/��Z��y/��Z���o���1j�����1k����YW�~hs��+��B�(��(��(��(��(����6���/����\�c������_�g�d(��1������;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���������;���_��?�k�c���������
q�����w�����^�;�(��
(��
(��
(��8��Fu��`�����u�'��#:���������k���_?���&''�AEW���EPEPEPEPEP�_�J<��?��k��������x?�����U�W����~l�QE|�Q@Q@Q@Q@Q@k��s�i��6�7�W�����W�|f���������F�y|�a���_������(���(��(��(��(��(�~
�����]!�O^�^i�o�=5����z����QE�(��(��(��(��(����;� ���:�����$����k���(�M��}�(����(��(��(��(��(��~9��f�?������������ms�����������E�?��%UO��Q^�EPEPEPEP@�(�u�/���':�z�	���o�����������W�8��������
(��(��(��(��(��)	��ZZC��{P�Z���.�qm�8�����V/�.�t!��L�ps����_-���vQEI!EPEPEPEPR���?�����[o�����/���q��7������7�������(��(��(��(��(��O����������'�B����CWO�^����Tv���W�PQEQEQEQEQE������?�[����C�<���$gY�z��G�zyG��h�?�R"����(�������(��(��(��(��(��[�/��Z��y/��Z���o���1j�����1k����YW�~hs��+��B�(��(��(��(��(����2�O��$m��;����M�1���:Q?|K&�{`g�W;��_�S:�<��(��<���(��(��(��(��(�g�I�"l���'������	?�M���d��R�;�(�����(��(��(��(��(�����O����%]o����������~G�����F�Q^���Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���������;���_��?�k�c���������
q�����w�����^�;�(��
(��
(��
(��8��Fu��`�����u�'��#:���������k���_?���&''�AEW���EPEPEPEPEP�_�J<��?��k��������x?�����U�W����~l�QE|�Q@Q@Q@Q@Q@i����t�~���������u���F��'7
�����W�W�f���8��(���9��(��(��(��(��(�U�7����t��=z]y������?��?�����AES��(��(��(��(��(�����G����V��;� ���:��v����}6�1�
(��7
(��
(��
(��
(��
(��>=���%�\���":�����Y���a��#�?���������yU>6QEzDQ@Q@Q@Q@����P��?���_��?�&�������Nt/���WI_�������_�$(�����(��(��(��(�9=:���<�t��o�6�t�!���c�����E���m�����=G�x���r��\�aET�QEQEQEQE-��~C�]�����������1L��c�i)_�c�i*�
(��
(��
(��
(��
(��
����^����j�\��/m���5t�5�kK���Gh:Q@�E}�EPEPEPEPEP^y���H�������J�:�����Fu�����zW���������"*|��(���((��(��(��(��(��+������������)�V��#��^K���L��E�}���:(��$(��(��(��(��(��+�~3�:P?x�&�;`g�����/���?K]��W9�0������w�������%��+�O4(��(��(��(��(��+����/�~����x�{?�O�e���?������(�((��(��(��(��(��+���������u�W[����������_��e�����EW�{�EPEPEPEPEPEPEPEPEPEPEPEPEAy}i�[5��s
����@�>��L���u[?L����8�-�Y?U$P5��������;���_��?�k�c���������
q�����w�����^�;�(��
(��
(��
(��8��Fu��`�����u�'��#:���������k���_?���&''�AEW���EPEPEPEPEP�_�J<��?��k��������x?�����U�W����~l�QE|�Q@Q@Q@Q@Q@k����F��M�d����������~3���H��hm�w����"���?���8qQE�s�Q@Q@Q@Q@Q@��o�=5����z���O�����H������(�EPEPEPEPEPW���������exw�A#���u�^��G��l7�c�QElnQEQEQEQEQE|{���K6��l?�Du������k����DG^_��?�/����(��|l(����
(��
(��
(��
(��
QE���~�9�����M]%s?���_��?�&���	�����6HQE�EPEPEPEPHzs��-!���@#�-D��.���������U�D8�.����q��5Z�.[������(�$(��(��(��(��*[o�����/�Km����E�b�N?���x�R����x�UQEQEQEQEQEsI��������:����^����j��k����#���t�������(��(��(��(��(��������g�����u������?�[����O(������JDT��Q_�PQEQEQEQEQEW�|���V��%���^S^���F-[����F-x����*�/��tQE~2HQEQEQEQEQEW�|f-����2�W��C����W���|f������s�������+���/���O��KEW��hQEQEQEQEQEW��$��6_����AJ������&��_��)M
�QTPQEQEQEQEQEW[��������������O����f��#������#N�(�\�B�(��(��(��(��(��(��(��(��(��(��(��E�y����?�7�^��k��7d�����q&2����)���g����.�7�i�"w����������9?���^e�iY��,�6�Z������z���R�Gu�>.�?�-��[��gU�h�gi�J��}�^���_����|=�pF��^U~��f�x#���	�tr�G�A5���'�p��:9�����;�����?��������?����O�Q�^�7�G�(��?�/�
������������_���������_���G�����M����
#�����������}������h�}���C��k����������s�t���u�B����������i�?P����>�i�?P�����?�D|9��x�T��]��>���?�*o�.��e���u��x�����	?���A������C����]��������w�@��������x��XyPT���{��]�c
�}�������?�E|;���NO����
+���?�r�.����_�����_V�>?����������9?�G�(���������?���?�M����������?�E|;���NO����
/���?�r�.���_������o3��+��Q_�������t���w�@�������y�������}[������W�������]������''����^e�@��7�jV�>?����������9?�G�(���������?���?�M�������>
:���������Z��V�����_����E|:?����M������u�B��������l�8y�#��r���~�K�}W���V�����_���V�����_����D�:��x�\��]��~��?�.o�.��m�/�������~!����4}���~!����5�(��_�/������O��������������d�j��������j��������E�
'������������������s�t{o ���K��������������������_��~��?�.o�.��Q?��^�7�G���������~!����4}���~!����5�(��_�/������O��������������d�j��������j��������E�
'������������������s�t{o ����nw��I�9�E�)�����u�8�+��=G�_O����v:x��7�G�(���/�����]y��7���{T�*�������?:7Q����������''����������9?�X}A7�e�������|��z�����~u���(���������?�E|;���NO�������f/��?��0n���p��}?�
+�����NM����
/���?�ro�.��/���17����p���������Q_������t���w�@�����������_���>`�=G�F��?:��W�����n�.��Q_�������t}A7������'������=G�_O����w�B��M�����
+�����7?�G��~�������y����KM_|���np3���}�����_�����vz��&����?�D�:��x�\��]?��{��j�o���i��{���h�L������K�
'������������������s�t}G���f���?����?��?�������=�����4���~��?�.o�.��Q?��^�7�G����j�o���i��{���h�L������K�
'������������������s�t}G���f���?����?��?�������=�����4���~��?�.o�.��Q?��^�7�G����j�o���i��{���h�L������K�
'������������������s�t}G���f���?����?��?�������=�����4��Q?��^�7�I�
'���������������_���:����\��������O����?����O�Q��^�7�G�(��?�/�
�����G�*=�J�9 �����i�?P����>�i�?P�����?�D|9��x�T��]��>���?�*o�.�����i�?P����>�i�?P�����?�D|9��x�T��]��>��?�.o�.�;�����?��������?����O�Q��^�7�G�(��?�/�
������������_���������_���G�����M����
#�����������}���C��h�}������k����������S�t���s�B����������i�?0����>�i�?P�����?�D|:��x�\��]��>���?�.o�.�>~���/�Mm�eu>F
��������?�E|;<?��{�����W�������]}��������/���6�V�S�X{��|E}��
+���?�r�.��Q_�������u��������o����y��_`���w�@�������W�������]�����&��C��g��W�����������]������''����^e�@��7�j?�y��_`���w�@�������W�������]�����&��E�o3��+��Q���2�.��Q����s��t�2�������y�������������''����������9?�G��/�������o3���+��CV�%"�d#�oz���o�?������W�������������������s�u���~���Z�����_U�%�U���C�W�h�U���C�W�j/�Q?��^�7�G�(��_�/��������}W���V�����_���V�����_����D�:��x�\��]��~��?�.o�.�m�U�%�U���C�W�h�U���C�W�j/�Q?��^�7�G�(��_�/������y�|�~�o�?����>�o�?������O����������
'�����������A�_2_�[���?�����[���?�������������s�t���u�B�������=��}W���V�����_����K��D�sqi1R�����=j��(��_�/������O��������������g���^���;F��6I8�j]��~u���(���������?�E|;���NO�����)��~,��)7���>`�=G�F��?:��W�����r�.��Q_��_������_P_���17����p��������E|;���NO����
+���?�r�.��/���17����#�~tn������E|;���NO����
/���?�ro�.��/���17����p���������Q_������t���w�B��NO�������f/��?��0n���p��}?�
+�����7?�G�(��g�e������������_���>`�=G�R�0�\<�����W�_������''���������������������_����`����?�����0�x����/�(��_�/������O��������������f���?����?��?�������=�����4���~��?�.o�.��Q?��^�7�G����j�o���i��{���h�L������K�
'������������������s�t}G���f���?����?��?�������=�����4���~��?�.o�.��Q?��^�7�G����j�o���i��{���h�L������K�
'������������������s�t}G���f���?����?��?�������=�����4���~��?�.o�.��Q?��^�7�G����j�o���i��{���j��un5kbn!������������u�B�������?�D�:��x�\��]8��d�7�Tr�)sm�����i�?P����>�i�?0�����?�D|9��x�\��]��>��?�.o�.��;�����?��������?����O�Q�^�7�G�(��?�/�
������������_���������_���G�����M����
#�����������}���C��h�}������k����������s�t���u�B����������i�?0����>�i�?P�����?�D|9��x�\��]��>���?�*o�.�;�����?��������?����O�Q�^�7�G�(��?�/�������������_��>��wo/��e#�'b��UpI��T���>���?�*o�.��Q?�O�os�u���}[N�����{;�K�4|�E}��
+���?�r�.��Q_�������u�������&��NO�y��_`���y�@�������W�������]�����&��C��g��W�������''����������9?�G��/�����P�������(���������?�E�;���NO����������o��>��|E}��
/���?�r�.��Q��`������y����������|E}��
+���?�r�.��Q�������t�2������������>H���U2:�6K��x����������''������������{����,K����7^k�����}���~!����4}���~!����5�(��_�/������O����������{o!}W���V�����_���V�����_����D�:��x�\��]��~��?�.o�.�m�U�%�U���C�W�h�U���C�W�j/�Q?��^�7�G�(��_�/������y�|�~�o�?����>�o�?������O����������
'�����������A�_2_�[���?�����[���?�������������s�t���u�B�������=��}W���V�����_���V�����_����D�:��x�\��]��~��?�*o�.�m�U�%�U���C�W�k�>1�����O�/�3mY2�����q^��
'����������D�;<?��s�u�i{Zn\4��>`�=G�F��?:��W�������]������''�������o����_���>`���������E|;���NO����
+���?�r�.��/���17����p���������Q_��_������?�E�;���NM�������f/��?��0n���p��}?�
+���?�ro�.��Q_�������t}A7������'������=G�_O����w�@�������_�����n�.��/���17����p���������Q_��_������?�E|;���&�������o�?��~�O�7Q����&�$�t���O�d���J���E�;���&�������������o�.������57��o�`�������>������/������u�B�������?�D�:��x�\��]?��{��j�o���i��{���h�L������K�
'������������������s�t}G���f���?����?��?�������=�����4���~��?�.o�.��Q?��^�7�G����j�o���i��{���h�L������K�
'����������D�:��x�\��]Q��������#~������/����?��?�����O����������
'���������������_����0�x����i��{���i�D�:��x�\��]��~��?�.o�.���{�������>������/��U�^�����as�E���-�
'������������������s�u�7��5�ta���|��;�����?��������?����O�Q�^�7�G�(��?�/�
���������i�?P����>�i�?P�����?�D|:��x�\��]��>��?�.o�.�;�����?��������?����O�Q�^�7�G�(��?�/�
������������_���������_���G�����M����
#�����������}������jdu�F��� ����Q��^�7�]���X�G����>�gj�!�qm�9�I$���QEQEQEQEQEQEQEQEs�,�.��H`���}������4$������������ZdS�}r6�{y9�g_��=����(��(��(��(��(��(��(��(��+�>:�����l</�]i�z����v�y(6�G���[�+���#_����(��i����^\����6��
N�6Gl��k�����������/d�Y���B�P����o?��|j��~�A�����7���������|.��O�G�K�BE����,DveL��Pk��Z�N��K�j:�b����Q�h�!�4�V�Z�a��n����n�h[r���	�SF����S��RM��n.�u��1��z��f�%�t���@��].�PIm�S������@��"MwD�O�R���R�]X��T�����5�?�;��Bo%��|Q0�5H���o�����w�y�)���~����(������u�x��w"������I�T��rp8��D���ojW6:6��R���J���)�a!H���~7i�j�������os���_�)��q���kw�o��/{�����V��q�
���(�o����D|A\����S�����vO��������a3����c�p�Q�x�.���?��<��z7���"��Y�y�1��7\t��A���i�g�N)�2����9@b�����:���~�����\P��	�U�Fq$e��]^��i�����^Eue2�I�l�����?��&�k�H�;X����n]�),G9�b���m����7m)��]j���!V\q��t�?�ku$���m����Y#S���_��k���>]~�S��Jh�E�f�
��#��+��=c��|+��o��/O���ZD�*�c��g������?�����^��i�j��KU����1�r�v���[�>/xW����5�[��l+<2B$=��������Ki..�Ha�KI$�T���W��L���s��T��5��`��!+Gn%i�`�o��`6p3����{q��3�U��������W�7�(%I�<�������!��y�	<���M�Z����F?�H����5��r;k-��tl������~�#�zT:>���|V�f���R�
8�1^y�W���^7�Y���-]���-�_�:��k7�ZT@Es{��$'r�YA=@�w�����M������X�i���0����)��U~
x�h�z�h��|Nf��?��Y���T�����"�����@���/jZ�v6� ����D�D��#gWe
:��E�]/��O���p���jE�cg���F��~"xoG��]���iV��R<c�dRT��1^k���g�W��5c<�f���I��4�7�|a�6��,�'���$����]F��Xi:\���y
��I����@���g��� �G��p�@�v�\�(���e��r��q�xoS������[)�7��mr�s�.c?#B?�u_�	�_Cio�$�#����4����@9>��W���h�kg��h��H���;.m��G*3��RR
�r
yg�+�Lx��Z^����Q�z���Ic1�����q����m����~��(��g��������Vq2J��8=A��Y��/��|4��jI�ss�,>��>�gY��<t�i�7�4{m&Yg��"��>��8��x�H�V������5K/�[�cc���K�c����x�J��vo�\4{r��b&}�7A������=V��i
�
�=
�G@ ���kk�0O'����E��O�5���Y�Zfi
 P�%���tG��kz�z^���%���������`~��zm���3�������^L$��H���b�~7���%�n�1�q�����+D��H������i"��^T�G��������G�k��-%� x5�b���.'��Q�%�l��009�S�=����iwu���Z��L�y���Mi|E�������.��P����=�����C�Gk4�1B����E�������b����In�����x�8�k����K��N��~��|E��Z�g,�)��z�Xc������L�&�w�\h�:�-�
=��*$1���������h����x�U��"�H����p�2�a�^���h~/������w��l�h*���)��W�|�,�]���%�=����ee��L����Z�������-l�a��H���4V�x�����v%����M�����r[��������{���k���$�����D�dU��g�����W�x#Q�l�&�f��xR�Z��W��[����HU8XB�B����]7�4OK�z�����e�{[�i|�y��3(�A�4����/��#y����R��n���c��@���l�A��A��5�����*��J�Gb=�E/��hu_x���k.������h�O��=�
��&�L���d��HM��$��c��R�� ���=��(��(��(��(��(��(��(��(��(��(��(��(��jN��wOe��`yi�5SU��=��p���CT��>Y^j���-4����������4��+?����������E�r�ax��+��A��O[�si{g�]wI���[6�F�r22������9y���^K�jz���*��Bb��hs�P�x��~��C�uH�f�wEFy	
��)<3��
x�Ib�������+G"R���+��O�ui<y���i���vb���Pm��DNp7I�=�����z_��\���Ksu,�+i*����I��H��uZ����!����"m��v���������~�/��}+W��Ko
X�������6��>0B��G��g����xo�����J�i����G����;o�iv������i���p�d(��N��<��g�����]_��X���[Z�m�.��p8��]��x?�O�~5��������=��5��7�S'9�������
��������o�L���q�%���fP����V�?����
]����������O������:I+(-�[
���q�h��>#xW�qZ>��$
y��n�H��PN?
��;���|�������	V�D��+��/����|��6��\����5���v�o���������)#�F�������kux���U���D���1@��������f��%��v����`�8�_z��}���~%�O�z�i�&��^*�I�����p�W�iVz��9���
N��)�#q���$w�r=���[xl�m��R��#���Kc*����'��@���q��$'�Z�v�>|����&=A'��|3��>0���@����)�I��:�q���k����|G��K����4�MyV��SXO0�H�b��c�9���Z~������mQ�n�C�m�"]�FJH>��=(���_������-eE�l�C���lS����i>'��Q�o���bW����:�:��k��<S��j���}�hv�U������[u����T=��Z��*��������zN�[�O�@$�U�3�@��~!�[�7)m�j�[��7-�#K)����;����u�7V���@I�8$g���=���W-�R��U��Q�'�.����Y3D������/J���Kh�/�������T��'�(�~|a��o
����[���J��E��fi ���|�y�����ZB�W�t�z���������H�Gj6|��������~	�LTg������A���8���O��T����O�<'|�Z��7l���o,�z�@HZ�����oi�����7����������c^u�2�
KN����Yu{�^t�Y,��mA����i�h_���e�"�k��uuc%
��t'��Z�/>(�B���m�����$��>�.3�~?��Z^����I.<;��x���T+#��V�����^�e7������%�z��)rcR	8���~���A��N��e����y�����z�PM��Q���uF�umaV�^�^gO��)��Z��N���4��z�/m��l�*���<��[i�-�y��^�x^/����^y���]G��
��������.��]���h��X�[�b����K��^;�PO!�m�C^������|�����Omr��X�2�i�8=k���V�C_?|���c���y��)�IZUm�1�����|=���g7�4O_��<.��PX�i`�9eQ�v��#H�����T�%2�]���!B���88#��5O�����5��5�kZ�����[6�>[aTzr����Q��K���C���'��M��Z�D�i������l�dE90���������
�W�N��u�.�D�O/<�|���($sV~��qS�uk���O�?h�}���/b�'���V���,v�P��:{���\�|E���h��^���%��:�c�k�>���^7s�@�o��1b��L�T?a�E����LP�`h.���J��v�A�G���'^#����������&����m3[�P�`I�j7�bfQ#-��s�Ua��������P�����Q?�����%�O
O��N�B��[i�!�G#ghar��c��_O�^�`������ E�[�<�ql��P?:��b���:N�����a!U+�<�}kr���������]~�����GX�l�!O>��O�v�Z���`m�X_j�>^�F��G�kO���b>
�n����Km��L@���P�^��i�R�_���*^Ie`��w$�c����������3����H����B�����������M>;u�]R���Y|��`���`X��#����|+u�]�:�R�xLhWS��'+(�uJ��CQ������+��m`B��+mT���x�����K�oX�:��zt��$)$-&�����j�������k��}��5@$��a��k����iw?	5��������e@�K'�����C���n�'�?������K1$��w5�W1���I�����+���(��(��(��(��(��(��(��(��(��(��(��(���l�0TPK18z��������-��5
B��S_�.4�[W��p����H����~7���7���[���GPc�_;������G��]|��zM�����������B�%x�-���x��>x���Yt��N>St"��Y��~���	�c[7�����=k���g�GY.4;�0���+�������3���	m��%z���o���$��I'9�cx���>$�����:���E��t��-�;��{V���X�����V��@
�xw[�~+�����([M2�)��o��-����e����~}gN��i���K�mY��"���g6z�sY�����������Z���=����+��i���A�s��Q������|H����~#�VH4�ehc�x�2�	B���4�~�&��n���^���	�xZO�C���Uo�>*oI�iz>�o��z��%��c�����`����|T�����-5O�
G��n�6����97��cs�J��w�O���u��nu�mn|�`��	��p+
�����������k�$��b�"w�0$#a��
s�����e�����[�9TeYy���?�ZE���BY�����"�Y]@���q@���
+�Z[�:��66����O@R}�s�7�����n��f���������E,���w���x7�:��������_�;��wg�R{���RX�GX��;H���.|?wu0���@������>����:���^�(5�Q-��7��I#.q��	�?�:���|���R;� �XJ�H���@��s%�^�u��<1�}N���mRk��E��1�`n'��=:�&c�>����sx���wy
�$�*��G�����1��\���?	X-���(,!c���s��9'�+'A���/\�k���[��i�A$nQz�VPH�W%gk��Kj��(���qe�2�n������Sk;o9g6�y��$����x���9i��x���X�}0]!�Ut�	H����M��}�k��_D������5��/������ao.�<�9��T�qX����@��5Xq��
O�g�G�1�>���.�;_��@���w"���IN����!�U'�_�>>���$�/������2�r(�(�{���*�s�����k9���t�IDa�,��$g�����:G���dk�6��9LcxX��P2:�u_xsC�$�5}I,�b�7���X�����9���V����	��R}?D��k�R�D�<N�;�p7�q���g����p��gKm�� ��B�;�9�g���6~>�w��D���h[Pa�6^T���Mv�'���<���E��h����k<�c�G�Q�c��|aq5���!sqo���u\�;\��-��x�M���<9�Zk��g5����[tlx�8��S�>=��A�I���
�@�y�ky�x�2Q����w'������|_����/���M2X>�
��EP���������5�:Bx��7����t�����G���ycn����>���j�"�_��_kZ������5�����>\���Q�����@�6��hn���t��{I���l������G�xc����9>1���m�����{X,�)���$�2j���4����]������].������S��'�-�To�wc�
y���c��_�_�|I�h?�R8Fgf;	��2O�
��X�S�x9��j�J��m�X\�c����E�v��(K�Lj�,G�8>��^����co,�����N%@
<g��`������7���YK���B���@�*��-��?h���H�����
�F0%U�u ����>�������-?��
���G�?6��]_�=�f��h�R���	 ����R���m+Q���\W�+/���e����)������>'����N%�:����g���+�Q�������v�H�S��C,�X�g��'������g���u}
�Ok5�hn b3�8=G ������t
7N��}v8����f�"b��e)������4~�������X>X������J��9��=���<3���-�������������1=�'9�����X�meI���"6U���q\m��?�jY�x�$O�G�)$�3�]T����t�b���C����D�(��Y�fm���#���
���7�Zn���D};��u8�|�G����N}h�mom��b����{i�<sF���������W�?��6�0E�c�[���n6u��>}?����|[�j�|�i�.�%tU�@7=zW�<-�A��N������k��P.����Ns@������Z�M+�F�tw-���=��k�?g�R��������.9����EPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEPEP\g��}��k���W�������m�u��p<w�����{�|m���GL�Im1�f^Q�>�5�������}����[_Kuw$�4;����A�'���yEy]���xB9��x�OMIH ��ZI-7��>���U�/���&�&��P�o�77���4���a����+�������������iZ�w����t^�p?Z�h�?����6��{�9��4}L^L%��P1������E�����{��g���r�@#��kF���7�`�c�������yh��"��l~U8�w$V��>�x[������m����|���EY�����
(���n|'�-'C��)�,���Hs���x��cx[����a����
�����^�\��cn$i!��:WyEy���>*�2�w�+����b��;�#�A�������[��:_���B[+��{�]��)'sBI���
��O�;����5s�����iQ`�����iG�i[�������U�
��~�x_Y��[���sX��@.}�����4����x�K�������si��%zd����|�����itmOB�b��4[���K������<�WiEy���ox�����x��H:���[D�"TO��������C�+o��������Ba���3F�\��<z���)���?��A�?����6�[+F[�����3�H�k{�~|<�������-��1y���q���WqEf��l����=2D���[tg��gB��v��[O�w�t��.��k��v��F��:��P��������@W���>1X4�x�K<s$�A�Z2�pP���G�5�H�4
�
���Eq~8�e���K�=F���(�K�s:��s���x���(��8�x�����xkS�L�t��	f��R�}�p9�������~1�E���x��6���������V��\�	i���n������|m��Z�K����,�*&V ld����G���<_%�����O�C*�,}�E%����{�*�:(����.�O����[����C{"��W��~)�*������	kv�G�<�N�n� t
6�c�|WaEq �� oOy�_��c2�,52�"U��O?�G�j�,��g��xu-j��Z"���C@�j��$����������	��o�-����%������t
�9�R��B��������gy�Gf���b��$�1�=k���<�S���~'���kv�l�������o3��h1�����|3���-\����V7,L�a���q��]�bF8��Q@m��?�������gc��C��������Z.�A���z�]sO��m��c[�S������f3���rrNMzmQEQEQEQEQEQEQEQEQEQEQEQET�����!�������G����<�kmR����G/o�i�@5�|}�|I�����-�����:97o}�n?:�h�=�G�����~-�>�k��ol-.����
�`�g�?��s?�X����^�V���_��N�qD�E[HT�Fy�$��+�h�B��7���$�;>[Y";��g$�8�?k�B��0���q��J6&���_vzc�k���

P�����
������[�%��5����|`��?�O	^x/�M�j3�<�w4������k���<�����'
��k�7��myo}	xgU���A����e�9� �>,�u�}B����H��q��$p��$��$��:(�����|_��`��qimda;��������K��^��T�����D�>���	we�G!p?\
�(�3��G������^!���x�M��h]U�r���>�S���a��x�������������������������-���?j��x�:u���\����-ZF�w��1�}q��W�����>7����TjI�Ai����.�1�zN���k���!�<M�|:��lT6�N� 2@�w�J������g�������W��f� G�������������/�+��	��G���"��P]	qn��ag,��s��5���>�J����Oq��1�,1&���&��#������K���%����O���ego����i���,���^F}?����G��<?��> �����L*�r���(�H�^����v�P��^�k]�M������?k�b��\c
���I����.?�K��o�/����?7�������=�]���z�k���x�F���Fh��b�HT��������j��xF
F�Q�N��j�&��������U�5�Q@�r;�W������Dz.�qoq:�K){}�p��>`
vtP�p�:}������6�E�V#�5�Z'�<{�������(�?)�����$��~�y8<���z���>�y����,�PLn���B�j:�C-�\���������>%���k�{�w����`�����z��x����Z�\__����������o����2q��*x.����t���y�����G�v�Y:�9����8�_����?�kpX���/�����Lmldv��ej�	����6M��&����0�/ad�iS9������P��������&�_^�3$��4H�B�^;���O
]x��:��a40\^"�I6v.78������E�+_x.�E�'{{�5���m��"�s����]���Z�~H~����r=v���5�4P#�����o�j:��o����STE�`z����\����>)�������jb!���k���������b�����6���+I��9f���<�b�������(��(��(��(��(��(��(��(��(��(��(��(���<K����G���uso�����Z��I'=�00��WhcB�*����@i��7�tO
������6����ik����B�_�������j�n�����o�O���;���T��8������Z���gi++=�	�J��[����o_i�5��qn����P����2[�c������/t��&��=���cM��`������(����.�P�^���CL��y��w�9dq�k��O���ET�'�4���������H#s��fm�c&����<q����������i�w�y��uQ�����@����|�hV2�
��>ZI6v)�8������Ox�<;��x[^��o4�)�{o:�"�q�2�ax�����w��sN��J�P�i���YGB����W�Q@y�O��|p�,�6�ic}un�������e_��r�~Z���e�9~!�^-�5�}B���.�ch��	(3�$�I�&�N��|_���T����'�
_���i$����/�H����V|5�x�5���]���H�E��h���Fgnx����@i{�oh�1�u���\Q�L��Z�p;�W���r:��|��_�'�YnmBh���w��${r�c�#��h����o|U�<�G���C�y����.���<z�=*����]������ k6�����,,�a1���%������P'��[��^,}�4�da���������K�^��Y�����D�6���	we���y���
(��&������^"����M��h]Q�VW^y����&�o�)<[�=^
GV�/�-����
�y��y'��k������x2�D������[�m�<-q���vF?,�iE��O����'�o�������h���m�A��!__C���(�<��S���_���������I\��m����N��?�~2���=����jW[��X�Xe���!�=O�z��xS�:�����o��huMk�6��m�H�P��I��=sW�����U��Q-�
i�ZAQ.�1J��cv5��@W���h^%���5�;5�����	h@F_� ���zVI������x�C�-�>�.��8�v��5�tP��|,6��|�����)g����31�:�oca%��m��e2Ej��HM��U�(��_�����������RQ2O�n���O�g��c�����M+�~*����F�c�Z3O
v\�
�z�
(��������4oj�[��H��
�G@;��KK��'O_����h1�*;��&�8�������W��@W������z]��3�/��&���m�������h��
(��
(��
(��
(��
(��
(��
(��
(��
(��?��
#51Amit Kapila
amit.kapila16@gmail.com
In reply to: Ajin Cherian (#50)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Fri, May 30, 2025 at 4:41 PM Ajin Cherian <itsajin@gmail.com> wrote:

I ran a series of tests using both streaming and non-streaming logical
replication modes with the patch. In non-streaming mode, the patch
showed a significant performance improvement — up to +68% in the best
case, with a -6% regression in the worst case.

In contrast, results in streaming mode were more modest. With the
default logical_decoding_work_mem of 64MB, I observed a +11.6%
improvement at best and a -6.7% degradation at worst. Increasing the
work memory provided some incremental improvements:

At 128MB: +14.43% (best), -0.65% (worst)

At 256MB: +12.55% (best), -0.03% (worst)

At 512MB: +16.98% (best), -2.48% (worst)

It's worth noting that streaming mode is enabled by default in logical
decoding, and as such, it's likely the mode most users and
applications are operating in. Non-streaming mode is typically only
used in more specialized setups or older deployments. Given this, the
broader benefit of the patch - especially considering its complexity,
may depend on how widely non-streaming mode is used in practice.

You haven't shared the exact test scenario, but I am assuming the
above tests are for very large transactions, as you are comparing
streaming and non-streaming modes. Can we see results with short
transaction size (say one insert or one update, or one delete) as
well?

--
With Regards,
Amit Kapila.

#52Ajin Cherian
itsajin@gmail.com
In reply to: Amit Kapila (#51)
1 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Tue, Jun 3, 2025 at 3:55 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

You haven't shared the exact test scenario, but I am assuming the
above tests are for very large transactions, as you are comparing
streaming and non-streaming modes. Can we see results with short
transaction size (say one insert or one update, or one delete) as
well?

Attaching the scripts I used for my tests. Yes, I used transactions
with large inserts. I will redo the tests with short single inserts
and share the results here.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

Filter Performance test.zipapplication/zip; name="Filter Performance test.zip"Download
#53Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#52)
2 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Tue, Jun 3, 2025 at 4:25 PM Ajin Cherian <itsajin@gmail.com> wrote:

On Tue, Jun 3, 2025 at 3:55 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

You haven't shared the exact test scenario, but I am assuming the
above tests are for very large transactions, as you are comparing
streaming and non-streaming modes. Can we see results with short
transaction size (say one insert or one update, or one delete) as
well?

Attaching the scripts I used for my tests. Yes, I used transactions
with large inserts. I will redo the tests with short single inserts
and share the results here.

I redid the tests with 10k small transactions (single inserts) and the
results are not great with the patch:

No transactions published: Patched code performs 12.33% faster than head code.

Half transactions published: Patched code is 4.97% slower than head.

All transactions published: Patched code is 6.70% slower than head.

Attaching the script and the bar graph.

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

small_txn_streaming.pngimage/png; name=small_txn_streaming.pngDownload
�PNG


IHDR���$��9tEXtSoftwareMatplotlib version3.6.3, https://matplotlib.org/����	pHYs��n�u>IDATx���wX��6������
D��Q���n�-�F�|���K4j��-�^b�+�X�aDEz�e�?x����eiz���r�����S������I$	����������������r��]""""""""""""""���t"""""""""""""""0�NDDDDDDDDDDDDDD�t"""""""""""""""�`����������������DDDDDDDDDDDDDDD@'""""""""""""""�:����������������0�NDDDDDDDDDDDDDD�t"""""""""""""""�`����������������DDDDDDDDDDDDDDD@'""""""""""""""�:����������������0�NDDDDDDDDDDDDDD�t"""""""""""""""�`����������������DDDDDDDDDDDDDDD@'""""""""""""""�:����������������0�NDDDDDDDDDDDDDD�t"""""""""""""""�
�D�ex��	�?����#""qqq�����5
c��)�}�������+DDD 11������%J�.����������I_�������~�.w��s��-�}x^�6w�\�8qB�<g�t���KDT���y����g��!,,QQQHII���!���`eegggT�Pzzz�V����K��^�|��1�-[�K���?�Y�f����s�NT�V��K���o�"88�>}BZZ���Q�B��]�]<�����o��A||<D"J�(�j�����Y������������hH$X[[�r��pss��������o!�H���<�������������
e��A�r�P�T�B.�����7n�.�����]�J���)�����k��t_S��0m��
k��XXX���C���+�R}�@'RA��A�H$
H999�����5C�5
���+55�/fe���{�.N�:�+W�����*�������Z�h��;�r��TJ"��6H�bbbKKK����F��]�6��i33�|.%����6H+������^�zZ�'���x�8q.\��������������eK���PII�RRR�r�J�����F�s�D�w�����G��G����S���
��Y�%	N�8???�|�R�6h��9F�
��[Saaa��i�����$[��e��_?���/�����$����������R�����t��������y�_q�-���q��A\�p			�nogg�5j�n��h��I�tr��W�uT"�������7>}�����Y�s��)�b}�@'��X���X���"44W�^���/j���~��]�|3�|�>}���AD>|�?�����W�5���x��%^�|�m���v���4i�}�"��!%%)))����<y���c��e�����cc��."�����e�r�2��RRR�}�v���IIIj�.>>���������9�A�1{R1�u�V|��@V�s��q��&))	[�nEpp0�����xBtt4f�����@��edd���K�r�
���a��Ls8q�-Z�4p�-<<+W������|�r�/_^����DDD����g��:u
���;�4i�����o�+V�5�&���(��������z�j���/���G>���H�����sG�\�n]vV-�LLL0r�H,Z�@V=�w��puu-��}�@'�G<��1c
�a*���{W.x^�Z54k����������Y�fA���!������[�B"�(�N__���HOOW�x��#Gb��!�������DD
%&&b���z�*V�ZU�i����#j===@'"�@HHf���7o����������)�d%&&b���8p��������_E&������������W����[��g���w���x��������ajj���D�z�X��k�"66?��CA������K�������rYBBB0r�Hl��I� ���1~�x���FFF��������b��)X�t)Z�l�����o���g�=Z�����FFFHHHP�u#��DD����;��]�`3)���t���7"22�����[W���b1�N�GGG0@n�X,FLL?~���o�+����_\o�#G��;u���s�~���5����q��E���)Sm��A�&MP�jUX[[K;�$%%���W�s���?���`�ksk�$"R�����(�>11�y�&��}+����/1y�dl�����iIDD��s��L�"p���G��M��iS��_�K���g�H$���>|�k�������t�������e����u+���d�}>t��B.�r���������7���C���
CCC���������e�`�W???T�V
����^���@�X�B����C�E��agg������`���8u��t���H��1��mS;�Ott4�O�.8�E"������{�B�
���CDD�=�]�vIw�X���g���+VTk��������I����K�,�����i��pvv\�bcc���<|�W�\�{��o��Yjo{��U\�zU�lmm�V��l�D�u222���z�j��[�p����[��K�eb�H%J�@���Un�e���������/G�f�P�d��,b��w��`y��Q���~�M.xnii�#F�_�~044T�:333������C�����n�:��u� �M_�.]�p�W�E�*{�K$\�p.D\\�t���/�~�z��1� �IDD:���SL�<YD���aCL�2E�|�zzzppp������0}�t���[�nUk^a*:���q��!�r��
Q�re�������WG��5Q�fM�{�>>>�**����q��`��#�Z��#Z�l��'���g���\�-[�����N��Sff&�,Y"�\�lY���_�P��t����W��y������-�f+{��9���������
6 22R�ldd�%K��y������)��c��Y�f���������$�\��V�Rk�����w�`��sg��9S�>���`cc�F��Q�F9r$���c�������RF*x������gA���L��%9�=����gOl��)))�M�61��O�s���4Q�D	,Y�D��kjj*�m�V8������.���k=�ig���r�(�.][�l��A���qss��u�0o�<�+IDJOO^^^X�~��H���l����KLL���������_��4x�������j=�2���u��]������W�^�={6������/c����2e
��o���K���b�X. ��I��A����t�R�c + �w�^��M�?���/^H����1�|A�\V��=��������@W$<<\.��������9�������z�*��������o!�H�k�.�2��;W��}�2e0f��8q�Z��q)���Keaa///���������B,���t�|2k�,� �����0�@��?�����p�]�V�����6mB�J��~��;b���(W�\^�HD��j����vMKK�� "��o����t�0h��<�Iloo�?����g��b ==�.���j ����O?��n�����"�(J��\����_K�������S�|y8P�n�������u��o�.X������{�����`ff&]������Gs}�������.].W�
���������!X'[vE��o���c|��I�n���Z����J�(��b�W�[�n��={�RI�l���k�N����x��]!�H��S�d��W��
F�T=�f���2e�������0y��<���z��!����;�P""���w�i����&L��{���a��	�`�����������V�Z�����"y��%�r���U����{�������X���t%44Tn�Y��=�z��h5@�A��t��U��R�^w��
�gXY���x���`�����QQ����t�:�|��X�\���}���q��a�����+����x��)���333����j��prr�������>BCC���T�Pm��l�=w�.I$<}��_�FLL���acc[[[������N���z�?|��_�Fll,233aggoooA
���������������`mm�������U�Q!!!x��	>�===����v��:I���/_���wHHH@jj*,,,`eegggT�R%�:A���#((�^�B\\���`kk888�t_o���������8�D"XYY���...����������������Btt4D"lmm�����5k����l���r�����'7uD^hs<���������DLL���`kk�
*�F�����~��������b1������
�\_������K��(Qu��E��%u^�O�>����x��=���`gg��e��N�:��W&>>^z����"%%������D��Q�z�|k���x��1^�|���(��bX[[�m�������~222���+�x�111HLL�H$���)J�(�r���R�Jr��5�'O� ::qqq������������}���g_c�V���U�~#�J�,�2e�F.FDD�|Mff&��{���_�����t�VVV������[���'O� &&111KKK�/_...:��������{x��RRR`ee�R�J���3����������|�����HOO���-J�.
��V�ddd���{����add����T�R:�Gq���)�}��III��������>S�re����QRR������HDGGC"����e������N��IIIx����y���$%%������(W�j�����SNE�<��g�`nf�9s�N��l��S�93!!������#bccannGGG��];_�I�Y7������%lmm�����g.u�:uJ�,��P���b����u��!T)S���������_��WOge���9�*UJn��*�����c���AAA����������<y"W���i��-���'�&������J���"g���ddLNNFPP��y���������u��Q�����

BXXRSSamm
'''xxx��Y>55�_���7o���D���
�K�F�Z���z��h������7o��������Dbb"`ee��%K�V�Z�����r���K�!�b1lllP�B��];�u���$<��_�F||<RRR`llSSS888�|��ptt��]����x�����������XZZ�����UC���T~E�b1�<y���P���H�yT�T	���022��~�[v}�����v�2e��V�Z:�O�����#<<iii������i25�"���x��^�|���8$%%���fff(Y���
K��:===�n�������]�t	:t�SyI�t�|doo/�.�aT������g�����o�nW�\9������W�!�����������;w���s�����q#�]�&��Q�L�i�F�C������}��9������FFFb����p����n��������_�~���������pt��U�Y�?.���M�p��)$%%���A�([�,���7n��m��Q3f�����������|���8q�\oo �1������}����j����������g�*Rw�����q��m�y�F�����Q�^��������7���CJ�V�Q�&L��F���OY��c���_|��A�v����^�:��m�.]���P&�q��):t���r�G6kkkxyya��Q��9s��\9z�����T������{7>|���L��X[[�E�5j���R���sq��	�r���?���M�6���V�jUL�2�����[zz:����={������9�7���S�������w>>>�s���H���������camm����=x������y������)Z�h�!C��Z�j�G� �\��u+�9���X��999	^��{R������u+��?���8���Z�jh���t��v!%%��7�;���Y�\9t���
��1�K�.����c�P�lYdff�������Cxx����*U
��G�=��.]���|O��Nqqq�x�"������x�������}��k����������6��P%&&�v���T�������Z�B�n���2����_��.]Rx�Dh��&L����!������������|722B��
1z�h��QC��$$$����N�Rx�OOO�?^����RRR��}{A]���G��z�X���;�KK�,A��mn�����������T������R�
�5k���;�KC^Qu��e���AAA�4�9����E�=z45z����;w7o����������
6��!C�����~r*j�A����9sF��r���S�N��!��|�����;�^�W�^Ux����&L�Y@������};.^����Y�����t���@:����������[�n��W[��=�;�j����{��][��u��N�&K���i9k��}}}�3[v�����������d�e_�Z5K���uKi�����Zcbb����������|||p��i��������=z����c>�<}�>>>
����#G�D�>}4��������s�~�:>|���4���D"��]�
B���������Ug���H�;w��]��{�����t����o��������TU���7�a��y����)z���#Fh��{�.v���k��)�f377���;Z�n��;�|fLOO��k�p��e�M[#����;w����^���C�������r���dllOOOx{{���K��S��;��7*��U���a��zem��u��m���w����7n��F��(��6�G����7n�P��fhhooo�7N���!!!��};._�����~/WWW�l�]�tQ�Xo���4�d��2��[_v�Q!SHRU	<y�$����� 9���a�����?�/_�u#$l��	6lP��O{�����ksM1"�H���c��;�v���%K4n���������_�v�!�H�z�j�����.44�f�������%$$�������������`��AxEF�����mvY:��G�b��)������U���'�6m���6�5w���1z�h�=Z�}ddd�������{s�Y�cpp0���q��5��������`��;Wi�6���X<x'O���?�(x���+W�����5�SPW>|��Y�f)�<�Sll,�?�3g�`��9r���MOO��y��F��z��&L��i���_�~�rO�:O�>U�Z�D"}�\�zu��7m�___�����x���.\�������p��9�������g�����>|8�����>��3vB��#G�`��e
������G����#���b�������;���_s�nY���7������_��ys���HLL~��G������X�d	�^��%K���yN�����M�>"[wz��9�����_����K�{�n�X�U�T�IY������7*l�R$$$!!!8u��`��:N�<���<w�b1.]�����c��%h���F�������s�F�)���\�z�{�����5j�}��f����?��.00P�0����:LLL��];A��'Nh@�q���zjcc��-[*��������_s�P�-33��?��������i��i\��&44�g�V������3g����s=z4F���>��Y��������q��U\�z�;w�O?������xd�w���1);?cA)���C�a��*�����8}�4�\���j|���c�l��1�Y +�����w�^���:�p�J@@���pss+���d�����5�P���&X~���b��;,��U�N�@�g�T�^�x!]���+���M�r777A=��������������q����kt)003f�P������={����[X�n�`���={�j�*���bbb�|�r��{,P+pz��,^�X��!����@4i�,�i��/�6m�W�^���S�n��_������w/�-[���Kk[d��b�\�{��U�]rr2v�����k��U�&��t�R<xP��$&&�����~�:\]]U8����\&
U����c����s����Z�y�����_��SSS��',,L�6��������g������p��\�t	�z���i��4�`��mX�~���_zz:�?�^�Z����M��q�F���Sjj��X�\9��x�3�d�'   �;t}m8i1Q>R��]��B�D�u��a��9
+�"����
��|���G���W��
���#�0������*�{��Z�
��/W��a``������s�1�=�j�7o������*����5`)
�+M������z��0a���fnn�05�X,�/���g���U6e#����aii	KKK�A3�X����c��Mj�G����c��qrA(����
p�����I�&���Oi����\��Pu� �|�2F��0x�����UJJ
����m����mddd�u�pww/���^��w�}��Q���\�5+==>>>�7o�V�v$	~��W��������;33��/���Wd=t�=Z.x���LJJ����s�����|||����>���v��������q��m��_Y�c===������o�����Wk?����`��	r�sSS�<������3�?��Fi�H$M���s����4i���y�=Q�53::��O����Sbb"��/<Wv<Ys�.X�@����'��SJJ������!����v0

�������k������_���*
�������B��A��sN�8��s�
�������%%%��O4������6l����������D"����1}�t�;B<|�'NT4Tt�J$���`���j~��fZ�p���$r�A��C�
���@L�>]a�<������k�a�����(����b�_��-Rk?��_�����?�8qB��k�h�@�H/Y�1�� �38�����}%&&b��

R{9edd���~�����
`mm������K|��wrsh���7�y��Pd�����x�k�r��iiiJ�
i+55Un��6�!d��*���w����
���B6H��/W�!?<z�����Q___�5����1c����s�N�X�B��N���{�p����/���jd�����z��k�0r�H���&m�8��A���amm������6l���$�`���������/44���Z�������������L�����~onn��3$''c��Y����x��9��4��}�+���cV]������#�~.333��)33�����i���Nl��k��\��?E����L�4I��[�n������oVVV:��XXX��LNN��{�����8�()j�Q�����[�l�sttD��}��Q#T�XQz�


�������'�h'''���~��={4J'���#��dFFF���7��o����K{/�{�N�<k�,�kccc�~�z����5���p?�j��[w��!��ccc0;v������������oc����`sv�b��=J��R$))	�������U�V���'<==����?��T��t��5i�J�R�0l�0�l�R��2>>^�p-}���k������WK_����!C��a����{DD<�;vHo�b��/�;VT�Q��5kwww8;;�D��c){�+W�`�������
P�N��RSS1c����COO^^^���;<<<�����0�8q;v�44�\��[��5}�X,���3�����/Z�h�j��I���������/^�u�'���:s�LAe���
={�D�V��H��������c����_��<�.����K��g������d����0x�`4n�fff�H$��s��m�6���c�P�|y|��w������s�b��>|8�6m*�c;**
�N���
���D��K��a�����_h�����[��[Wz��~�{��<l%$$`��U�CBB��~===t����w�^c��-��o.^�(}]jj*f���}������R�Jh��<==Q�J�)SFz�g��{��
���O�.���c�W�:v���g���Y��zzz��o�E��������X����)EYRR�-[&Xgoo���Y�fptt&��������{�����.����`���r����7z��)��"����S9rG���\�h�t~@M,Z�Hz��Y�&
��
J��qqq�t���_/�F�>}]�tA�
4�_q���'�4��R�����A�h��)�U�&7�s||<=z���O��HJJ��Y��s�N�=�+V�(���}��w������Q���,Z�g���366F�=��U+���I�Eb��_����q��%��yS�}�������D"���	��������j������D"��'O�k�.A����t,Z�H��t111�0a��^ ���];t��	�j��(SSS���������c����4i��}���+l���B�>}P�vm�y���c>|���D"���k5J��
www899I����q��9�F�����e�Q4�D"���,,,���j�
�+W�����z���W8�4�
�9�t������C���(}��YG<x� \\\��"�l��h��)���ggg�+WN�(���8pO�<���y�&�l�����~���hddT ��� �3�����K�
�b�����*U�===���������s'������4��=[��:9�X�B��M�j���OAz^�D�W�^���s��{��������?���;w������ 9v�"��T�Q����������&44T.���t��OU]���k>~����d�n��o����*U�:FFFb��A���+�v���,��HMM�O?�����D"t��]�tA�5��y<���� �������k�����1x�`�m�V��#%%W�^��5k����{�nt���*UR�����h��	6lT�XQ�������w���;w��_�x�e��a���y���t�h����D�����qcT�ZNNN��ett����Y����~��W�[�N�r>|X���T�R2d�7o.��������������}��[�*m������������~�z���t�����D����#$$�n���+W��p#�����������
����zCZZ�?���/c����N%�/����Z6"##1q�D��m�
��_?4n������;���S��s���W�9�y���L����5k���5SZ��%K�Zfu��b��5!!!��NNN>|8�7o.Mc���'\�x[�l|���V�X�_~�E�}��qC:0���
D�6m�uD�X����c����N����X�j~��w�����{��*T�����Q#�)SF�����G�x�����������������_d[Qaa�(�$$$���-]��\�����r=8�I�&)��Y�|y4�;w��i��=�ccc�p�B�Z�J�2f��-Y�$��Y#��!����$���l@
�����n`


��!XW�T)�[�N��`ll,�yo��
k�������OX�`�\pE����������}{�m�����o_�����W��/_.7����R8:t���"!!�����w���C1q�D����0a*W�,x`��>�>}�����:uB��-U��500���\\\��_?A����L���*��F���(DEE���.T���\�r3f<<<����K���8u����[�������7i����������!�W������o��x�����
���������
����:�fcc#�Sh����g$	~��7;vL�T��

�[W��R+W��k�<x0&M�$���������c��������/x���/�4i��������s�}�������Fu���a��Apuu���c��cxx8f���[�nA__��OWx�;99���~B�R������c��i�@�:�+����X�b5j$��������z�������?��gp\\/^������G��M1z�h������K��={���e�����������uv�~���4h��������sg��S���W#*��+�m��)�lmm����Q����3�s�edd���lc``���M�6�mE"j����5k��o���)S����b1����a������n4�9WNVVV���6l�#F�����"�d;�������:u*�v��4�K�v�5B�F���OL�2E������8{���e��%�u���A�D�j�wr:r���H�5j`��e
zE"�T��*U��[�n����\�]���O�
�����#���C�50�|T�PA�����{x���V��rs���e����%Kv�266F��
��aC�<y����^�w�����[+��m����N$����3g:u�$�.�!���
m����3���&hP�/]�t�6DY��5	��={Vp-�^�������G�zVVV��m������������������\�i,����1s�LA��j��X�d������9Z�j�V�Za�������+W�Y�f*T���h����=^V���Q�|yt���7oL%����������e������T''���T����������Mkkk�_�^��511A�����Y3l��7n�jD�����\���������A���'���P�re�3�:u�������w��a����9s��e�MFF�\cxAw���l6m:���R�*z��R�~��Os#��T�Sv��6�������)�3������������b�����?J�(�Z�j���
����^�z��yMdgl������+�:��D"xxx`����6m� [���;���	�X��U�b���r����	����mZ�u>�X�C��:L����x�b�n�Z�(���K�}��h��=�9�E�I����c����>�Dq��6���Kc������V9R���-[�D��-q��U��5K�|��-h<uQ�sp��M�h�"������4i����c��)� ����1j�(����'O
�[�n�E�)�^OOe��E��e��E�����u��������i�������
WWW���S�N��g���`��m�;w���H$��5K.x>|�p�3F�g277���'<==1j�(�e��n��:X����QC��]M�������������*���d�����:t����'K� �xh��9Z�h����;���Y+W����E"����5k�`��y���������S��I�={V�)�V�ZX�n��v�R�J�T�Rh��1&M���w�j4P�\�S��$�>�p'�'K�.�KY�h��?��SP����o1u��\S.�����?�@�2e���^�*����H�U�V�C���;)V����f��\{�
6L.�z��%�R�f�:u����&��/�?��C�C���,X�������I�T�$����\E����������h~W,]�+V������9s��:'p��
��gO������|Mdd����&M���?��u�z6GGG���]�v	���
+T����?
*mQQQ8z��Z�R�����{e��?�}�:��~P�upp��u�#]�b1�n��q���1o�<�
�u����f��:th��d�
&hh����������smX������T�/_�������sYFFF���_����0\�vM���i���
���"��?d��:1T�ZUeP���Kr=|g��%<�U�~}��r#����O����0e��:�F�e!!!��c�`����\x'''0@�FH777�ys!�Sjj��C�uln��A��c%K���\377��5kr��w��Qr�r���yS���4R'#J�N�#�%����SDD�\C���c�����6m���9���J�N�
�w����w��~���CQ�@����gO��k._�<������y���{i�&m�u��U�}�<yR0
�|��X�n�Z#��~��I�SSSs���s�����s�����Q��ov'RU��y [O-�:*P���.\�k�����+Tv>�����1c����ddd��?���:u*�k�����c��U����c���t%44Tn�U��@6��m�g��i3��*��O����Mdxd��j�?E�Q����[Y�{��f����t���Y��c��u��0`�/_�[�n�,�����Uf�200��Y����_����oacc�0x�S�%�F���&��Exyyi���{����,����a�5��������u�h*�f��	���?SU�V
��-S�����P.�%�V K��<a��;�����Y�f�v��0a�F�6{{{�������?���p��._�,d7n�F��^�z�BJJ��������;w��������z�j��^�iHK�.�5k��l������?�(�&==]e��l6���Gk4��N�:jg��&��H��|R�t"���3~��'�Fccc:T�.((H0�Y��e1q�D��emm-pQ4rK�^�z�:�Y���v�w���ri�sk������>}�h�E~����l:t�[gdd��S��������CCC���������Q#xyy��m�^����=S9���������0w�\��lIJJ��Sdu0����������L�"xH�������e�8���
R,--s�}����c����|��\�������j=�+:����1j��\_+��@?~�~!�?OOOx{{����a�����y<��l�m�u[[�\�y%;�E�������[�\�Nt��]��f�X�I�����5���{��m��5�����b�8��0-H�.]��1c��0���{����n��puu�.?y�D����r��q���H�h� ��a���5bG___n�tnu
������5����|��e����G�
�;���2d�Z������S+h�D�h���`���jBBB�###���QtE=0U�$�\c���35�1v�XA�Q����c8��pq9d��YG-���C�	�;u������?h<���������������/_���Y/55U���������@�����{Ud�u=���@m�*\V@���=K+�gq�-�M�<��OW��@"����g��w/���.]�`����)Y4��E���+S������#��g�m�V�?~���<�����o��yN�����S];vU�U�9s�Z�x�R���sUm3����-A����4�HjE�m�&X���������x���3��"�?���Z�������\�KY�&MRk �����s����0��d����Ae�:i�t"
DFF���r����ol��S�LA��]�R�Y�e�eSq���C����MA I���s���E"�H������\#��#'�w��k/���������\�rr7����!��l� ��k�#5�����&�~�T�"����,7�vN�=��w���������*;���G���5z�
*�;{��E��J5�3P�����3���IA���n��	�+���r���_��������
1�#)�9�r�����!z��!X';����Z���R;v�����6.�2UBB��CV�����m�~����i��������H$���L����1c �W��+W�(�?���a������f��!w��T��\����y�gffJ���'�{W�6m4�U�)}}}���*���������T(O�``` �l���)7�"�l����k�)�E�Q�'O�T�����c���-[*m��
Hj���K����\�+VD���5z��y@�����Kg�J�*	�������y [O��zBN����w�
�i�/�k�N������}�j�L+�9T��u�>�i�b���f��-�2���u�VQ�!m�*�e��_�S����[���:t�z�\�s������o�U�N����g[�H�v6SSS��k~�KYXX��<{�,_�/�.�85���/x.����(�������]��k��-X�Y����my�I;LDD�������5�z�(�mWl���Z��gk���\���{�bmm���/@��Su�.�g5��P����kT���1o���K���H�q������XeG������������/������hkk���L6X��������m�
�����"""�N����<���][�TT��@�:u4zmN��)�����/^ ::���HII�K&�P�VF������r�����g�u�����	�EDD ,,L��c���O]��@V�7�!T"����r)����AY~]KNN�kD�-��,3334n�/^��������_������t��d�����HU�H�����U+��?~��P�ti��#%%/^�@dd$�������kG�s]������l�GGG�v�y����{100�u�	Yu�����
bbb�������������d�W��AA�9���*W��U�Vi������x��
�������lm�}M�����}e���*U�h��Ivd��c+00PP�[���@�u���>Z�n-X�}=��e����E����/�^�-Z�\>|��[�n�L�����3g���j���.;vU�VE�^��\�����\�WM����M�������rv�~��������E�_���������d���s�?}����L��aq:

��
2�Q����=\K�(!wn��U�Vr���Q��H�����	���� �6(s���(X.�������)�?�C�u��UQ�~����G���:+����F�}��BV�����O?a��)�z�*n���;w����������W6l|}}5������v%ggg�2a8882i��)##o��Exx8��������#�X,������(��H��������w�������D�Yd3G}��A��r���s���� h>w�\,Y�D�+���/_"66���HMM�ks��&�z���Z[[k]�-jd�l��+����M�6�)>5�^�vm�:�j�����*���?�@�%���LQ]+11Qnd:i�t�|������'+�$''�5�j����"33QQQj�:x@.��h�hn*U�A�LHH�Zt�H��Q_�������
�+z�&�w�y�����4�
h�177W{.�l���l�K6�t������������;�Rh����###�T��U�
"1�|�R�������9\�fMA=�y�s����@������T�XQ��W���055t�x��y���?���c8�<^�|�q�6�����4o��J����d5���5�+W�7�|����z��Z���U�RE����s�����u�y<���,�����gO�;6��C"�����O���k���>����W��FSi2�B[�yl��+�����W���Wt�|�����cff��|�@�}�b�����'���:t��={�H�?~\e��������.�*UJ��5k�D�5���b1�,Y???�k�M�4�����iou���L��D����5
�������)���:����X�8q�������5N�.�H������Pq:LMM��UG
�93g�_@�C�:4y�����+SSS���?�����}_����X,��cxnd;M�w�Rd��r�����������KKK��;�����)����6Ww�E��P���^^^������x��)�<y�;w�   @a����������{�������5�&C��4�mZ���SRRp��i�={AAAZu�(.���.�8322p��y�={�o�������Q~>��������{���^�zh��
�����hgU�<y��������Zu�V��������h���-7:Z6C�:d�y�d��4#��uY����cG���J����(�;������B�F�������RQ=��;t�t"�D�����\]]��E��QQQr=�V�X�������5�M�@�.�b.�HA���)�-,,t�@���u��I4y�ly�	����c��5�����4o�&�z@�ye�e��9������V9�`���:y_��>/���e�xUd?������ �kR���{����}��AiM
����� h��=�s�H$��m6o���
���:P0�ccc��7��������������/LMMQ�F
xxx�n�����T+����������5S��eqabb����f���]�6��i�V'�W�^����x�3Y�����aaa���pmizliR���W��uK��][���������V�
e��)�:t��U@�|�2���d�I���S����~�
#G�|g�����m�m�###T�V
���D�z���(���=6��;�pj-M�V�;z�(V�^���XRR��zq:J�(!(oA�Q��}��]�M����N0\�c;99Y��x�H$������6��U���;��S�����g4���RRR4���\611�H$�7��n�F�>��o�.KKK��W�����A�����3g�`���r����0�����
S��5}.���i�\����<_�rK�,�s�����gym�

������<�C�������������������H$�}�6n��
 kZ�Z�j�N�:�W������l�2�:uJ�m_��m�v u��u��U����
�H$j���z���������_~��9s�=z��=���	KKK���������C�Z�t���t�t"
xzzb��
:y/]�d���Q��[�������}��SAJOO��i�4�^M���R������J~���������V�Z�l��~?�>tg����~�d�]]}?�z%��?G��j?9i��WP��z��a����?>�>}*����d"00[�l���%���0t�P���}?
�x,L>>>:MY���S�?^'���Ng,[Fm:�i#?����W�^�tu]�O...�^��t�Hjj*��=�p:���h�������r����c��������������x�����`bb��-[b��!��-n�+X���i�&����d?��G��<(_�� ���7o����U�M�}]���:������J��\��|�M���M�����4��LC�Eu���?k��U���*���������Q�e����[h���={�D�v����?�u@<x������\|��	��7O'u��������3��70m�4�\�5mO���o���ppp��U���+W����+���]�tA���s�?���c���:��Z��-{�/����$[������8��>�b1�����C���b��%
���������q����{��1p�@����4��I�1�NTH���Iy��F���m����666h��<<<�����%K���FFF�^n������kA�����R��d�*U`hh((k�����q��	������������J�*�T�R077����\�uA�a�+5j����;q��
�:u
�n�R� d=�>|����Q�0b���&P�5����/��"��P�fM�l����ppp���=���`dd$��}}}���������u��U�v����
��N�4�g���Q�|y��� (('O���k��p����={�������)S������)�c300P.xndd�V�Z�^�zpqqA��%aii	ccc��p�.]����|)sa�V����_�rjj*BBB�JqN��8��6\+JW]�T�XQ���hYE�_M�Z�M�
���/�DDDh|~�~>U��X�� ��6���k��"��O�k�Q�E^YXX`�������������

-�#M��{��
�U�H�&M��a���V�J�*��IF������.�W%66����������C��MQ�F
�.]����g������'Nd�5��C�n�����?����w�*�'������{�����s��ys������K�.�v��������C�R�`bb###������;V7���F�a������q��(���g���a����>}:�w������BG�/�D�DQ��W�~�=�d?���������(.lIII��m�`]���1{�l�����K6�m~��hee%x����?��iS��'/P�V-�����dt��y����_�����G�
�D"�h��A,\�P���E�\WFOO�7F���d�z��]��w���CXX�`{�X���a���r���Q�t��IAj]CCC��7�|��Z�/��|����9e��!C�������}�^�tu]�o���X�j�t������WrsT��o��c���Q�vmY!s^e�uK$����������5�WQ'{���5+Os��Fv~��U�b��j��V�^\��OOO�uw��)�zA���7�=������=��.]Z�����&�
W;
l���dhh(�^6����Q�LA�744T���-����e��?����B,,,��kWl��U��K�o��Q0����V�\	�^_:�w{��dy�����e���d_\�LLL��sgt��x�����,�);66��O���+��I��{���\��A�a���j��irl����y&����233����q6��#{z�����m��E��m�������K���w����RRR0�|�(�.�>�������s�)�n'�%"�)�g� ��+,�mF]��b�^��1�{Qt��5ARGGG��;W����Z.l�A���0����=/��9)�6&&F0�'���gqqqZU�e5���W��[ff��HE��������X�x��s+�s]]NNN���~��7=z������C��T��}� ��M�#��9���K�.	��
�v�(�c_�����P�����W��O>|�j�dA�����B��-��;&X~��1BBB����������~��-�N�:��_~����q��q�?^��p��i��y3O�*�d?g~�kQQQ

�.�D",]�T��yFF�����|xxx��e���R�����WTT��is]w�j��yL�����i5�+W,'$$(�;�����r�
�e>S��X�=������r��T�e�m2��~7���(��P�fM�u���������s��<��wP\�xQ�<u�T�2������nnn<x0V�X���6l�������X,������._�,����[�'OV{0�&��l�VQ*��HQ��6�(����E:����>�U��~��a��E8}�4�����wo��g���=c��Q�\�t�"�]7�kkk�����v��8;;���3���Wr�Id��k��!����Rt�T�5j�,'&&
�X���U�
���9��C�F���~+U�$��D"��U��q����o���x^��o����V�����s�r�f�4�\��umU�T	�&M���;���b��QZ����/#%�!�H��c_��\T���:u���}�W�TI�[��[qT���*U��;���x���F������ktAv4����!����������4��"e���w�}����[n���3gt���@v~��<6_�x!��Y�V-�F>}�Tp<�R���o�^�����{�n��� �3e�����5y���� }�X,��s���+W����e�D�����T�jU�z���j��4h��r)"��9:\<\{D"���Y�����?�(�����g��	���n��o��:�����K�K��(0��MoR�ddd:��]�vj�^,��2M�D"xzzb���X�h��o���
�=���:t�h��<����>x���v�������<��<����4��S����F��5k|}}A���D��H�����T���r�D��a�����W�RI
����`9((QQQ��lO���K���!�e��~���4r��
���.]Z�a���S:���9y���"YA-Q�����n�����pE��MMM�*���As����7n��^�b�X��������J�RxGGG�5�^�rE�����+�g���u��@V�#�|d����^���r���S)S��=�5����������lopM3��]�N�>��{5���g�����O��/�M3�\�rE�@�.5l�Pp|FFF�������4�={V��6���eoo����)�.w����;w4������s����o�����d������Z\��������=��������bbb"W�(�m"�H�3�l'�����@n�.MR�GDD�����u�YHt�U�V��>���{j����s�ewww���W�.w}����x��`���15j�t���[���9��{��������&������s���:>�&00P��K�*///���	�)�+��&33��������[W�����%[g�u�OEj��%X��]Q"����8�+��]�[����YM���(�	i�t�B��m[����'�������������MII���G�����Iv��&s�>|��@F���l���G�j�0�J��M=�_�z�Q�VA3f��C��e�t�[����X�j���5k�L�|��������G����D�9��D���W{���t9rD�N6]6���&�zDD�\���F6}������\c��}�4����{�+V,�suy�����i�)JvD�6�V����t����������������Ll��=����E���#G4J\Z���G����SY_�rE���B�
�S�N��Gv��8�h�Z�P�T)�r\\<�/����(11Q�r�����=z��		�_�����H$X�n���lA>gZZZ���C�N�}���h<��l���={���H^�!��HE�����o�=����#�����u�]���//����Cj�6!!A�@���l����cj_>,Xn����tK���o��a

���7y�?��b���[�E"��Qbb�FAS???]�HP�
!/�0������M���K���m���A>����\������]�([����\
w��,n���&��E���
�D��i���^A������_��;�(�������;w���};��y���}||�z����_g�+�r6��LHNN���s��D���A�8..s�����z666����`��������s�l���8q�`]ll,F���<q9�>}C����f������!66Vi�]�����+X��uk�{�'w��Q;3��m��z���p[������j�b�s����(�����-{(���_?�r@@N�<��>����z=�����/���'����{Q#{�Sw����7�n`�I��	

������� M�X,�/��R���F�,���?O�!U�+�w�.����[���C��=q�D�v�U����������ucU��.�v�(�#�100��a����_�Uz�l��M�kQPP��
�K�,�����vL�8QnT���;���j���?�����e���KA?g�f�9q���#����O���v�*��}�������=d��s�l����I�
$W�XQ�,�H�l��\_��;w
�
0@��p��h��E�"X>s��Z��W�^-��lmm������:��&��9{���o?t��\_W�����;��+W��3@@�\6OOO�`VqS�dI�������m����NO�%���INN��;w�z���Gq����*Z�I$�GP��� ����hdd$�/_�q�d�������-[4~Ud?gAt�n��=�����b��-R��KHH��+�j��Yd������� �njjZ�;�G����'��b������P�=$	n���)S�h���08PPYKMM���s�T���S���M�6����.����{�n����L�0!OA���(��k�0u�T�GA�}�V���C�
�>���#Gj�����K,Z�k����u����o�R�������]�4
�<|��������U��*Y��\@���c���Q����GL�0A�CW$�5t�~��w��yS�6����
�Z�h�t'OOOA����0��	�������?��p\L�0���F�X����F�(�w�U�Vr����s�~��g�:��`�������s=>�_��3fh(����SRR����edd�q��	�={�c��Q��S�O�>��X]P�5k&H�*�1s�L�G�FGGc����4i��m�������u�k��a����G�dG����c�����L���5�v��3.���/�~MXX�\�M�#-Nz��!Hw�������k�����{�^�s��Q���������X�d������,^�X�)���y`aa�+V��4������5J����???���+��������%��>}��������jIe�]+�����1C��"7�t�F����%�xzz
~�G�����D"��#Xw��5���(}MTTf��)�l������o������-86333����O�q~��!����!C`aa�����+�n��	�m��Qe}����X�t�`]�&M�27(R~��������u���K5>��\��i����Me�:�#{{{T�TI�,�H0�|���)}�D"��-[�i���(�WOOO����u��-��M����^��MBBz����{�j�Q����Q�zzzr�������3gr�����b��1Zu�i����o�~�z�_�^�@m@@��)�d�]	���WLLL��w7o�������s���c���r~d;w����c��M�����r�1�}V�{�����~��rY&(ot�
��4����)S�z���{�z�B���uk���	zf�������x��9����+W����� ��R�/_S�NT�������c���������#�������;w�`���r�&�,YR.���suuE���p.����o_T�Z"��o�����k�.iE�������U|��5
AAA�}��t��k���[7���-Z�@�j��=�322�����/^��;w����td���
�-[�1c� 55@VPz���h��1:v��.]Z���������s<x������a[vD����3��������x�\�{��E��m��IT�ZVVV��K���x��%q��y��m�2e
���[��M�6����<x05j$m4��s��m�6���c��z��y��EZ������s���b��I���z������C__

��}�p���k---��O?)}o{{{�j�J��o��/^�����Q�V-�9���{������c>~��x��@�������;w�f��h��%\]]Q�Z5�|����x��9��;�H�] k�`�6m����,X��C�Ju���0e�t��	=z����+ ����3=z�<��D"��??����w=z�����
����4hF���m�J;@������{8r��4]���<<<4���EAg�������pss������q��/��u���wq��	����`���={�D������*��"����<x��/_��7P�D	4H����`��a������X�x1�����={�n���T�������FHH?~��W�"((b�8����'O��k������L�������G��}��<�X�'O�����8z�($	����o���t��Ep�m��]�`.;WznRRRp��Y:tU�T�>+T�V
%J���M222���+\�t	{��t`333��9����!���2DT�����3����n���������~W����x��9���q��<~��Di:Ht��M�j��������������H�����c�t4���#���:��yP�Z5�Z�
S�L�z�q���M��i��h��J�*%��$	>|����`����p�������ihh���gc��Q�s:&&C�����sgT�XQ����o���O���l��HII�������?��]���.]��[�n�[�nh��9j��)��������_#$$7o����'}�������h���������[�n�ME����`+����������C�\G�~��78r�n��%]�i�&<}���G��5a``���x\�p�7o���7y�d��_tM__?��#��/�����c���6l:v�[[[��b���`���r�*W�,��I�1c�������Ojj*�M���}��O�>�P������G�b����L;����:u���+.�Err2����}�����M�4���;j��	;;;��Xff&BCC��'O"  @����k�r~���W�^�6���2#G�D�F��7���p��
���Sz�������N�����^�z	F��|�������ys��vRR�o�>���������!���P,_���'6l��M��F��\�����������#r�1-Z��K�dM�W�dI�������:u*����n���R�J���Cff&�={�������}B�v===,\��7o��s���_�~h��1��//m�KLL��g��������5j��{p�����$T����a���U�Vpvv����`�F��%���#7����W�{�?���`6��5���%�����K��y�fDFF
��G�J�>,���������Q�Ni����Y�y�������q��)?~\��T�Z5����f�������t�"���HII��u�����T9rD:���������,W={����o=�SRR�q�Fl��033S������/P(�r5c��;V0�����8~�8`nn�����b�k���1c��|�1�W"�K�,��3#i����y�fl��@���D�r4�2nnnX�b~��gA��k����Hddd ))���O044�������#7����#j���aii���������V�������X�b&N�(=��� Y�ZZ����]�vU+�_q���ooo�Z�
���8|�0>�Hsss���+<n����h�"��w����{�"W�\��+W��gRR�\&�R�J���/��I$<z�H����333���fll��~�Me��������_�Uz�J$�8q'N����>,,,���(w��.�O?������j���w�����t]ll,�/_�������"�H�|t#G��y*�����S�QM���_����n����O?����T��������g��� k:}}}$$$�iy����f�L�>]0J����T�zzz�F���$�����Z�t)���{�����s8w����4{�ka����,_�\@��]�t���{�x�B��4�{�������Y�f������+W���O�.��u��}i����*������w����K�����wq��]i�+%%E������/��i�4�_q<������7c���������r�mLMMadd����S���U>�s����M�&���a��m��m`jj*wO222����������C����X��{�J�%&&b�������055ERR��Q���S�N����K�4����c���jo�x�b�k��I�t===,X�#G����>V���ajj��9j���y�_�R�R�^=L�:Up�EGGc���X�r%��������>`oo�����������-��	��q�X,��I;���D�7o�����_Q�-4"7j���B�:P��M���z����'N�8!�����`��Y�������Ngd``�����<E5�k��%�7o.���������[�96s�L�z���t\�zU�-#�m=--M��/�2e�(�`bb�i��a��Y�ub��v���]�`dd$����O4h�C��x C�%�v�ZL�<Y�����[�3Wv�355U��/�}�~��W�rJJ���===u@�DX�x1&L� x�y��f��
 �8���Pi�������"� �0e``���+�YYYa��yj�$�`P���Z�n����S���
��5kP�\9�OKKCLL��������||E����1}�t�nJ5\\\�e�����w����k�����-##���ro���X�n]��o���
�����/7�/[BB��
I�9$�i�������������U��jll�QC��D"&L��-[����'33SZfe4h�;v����U�R%��KHHP�`ghh��c���_U�w(�
�q��	*�b�qqq
����j�>�P�V�X!�P�s���r
�NNN����"������N��t��������{yyya��5
����L���)l����������?�tg��ih�����%%%�*���1v��\�k�T�X�/Vx�i���.��q�����F*e�k��5�|����m�u���~,�H������x�
��u��V�vm�^�Z�R?���TOOc��-�Q����h�����YYY�U�V�������A�5���K�,A��5�gqT�zu���S����]���"���e���`��U(S��������{v+Q����/��yE��y������wc���*;�%''+��d������q���\�1������5K����=�����/G�����GN"���O����agg�p���T�����;88���J�2��Q�F���W�\�hz��bkk�
6(�?4���,�H��c�b��)�Kv�S4m�T������g�Vx�)�V�\7n����Fe����_���KKKS�Veee�+Vh��_���������+�[����&M��+`dd��b###�\�R�td����r���k��2
�8�?��4����r����!�����,F������=<<�e����l^^^�6m��g���4�����O�l�+V��f������o���
*�{v�SY�<����;b��qJ�W�K�%�y�f�#��[�����O���u+��������(m��R�
6o����#�{��	F��m���}_�@'*B6l�����38r�>|���&e��A�����eK4m�T��qa���?��m�-[����JS�����Z�j���:u�T$��///T�\��������t>�
*`�����'D"������300������_?l��
W�\Att���E"\]]��];�{��+W6l��;w�g���};��VVV�[�.�6m
///�����Z�ja����4�������������Z�n���;�B�
j��t����u+��?���w����J�5+++�l�#G�T�Q�K5b���[>>>
��Y���o�q��i�Y����v����/N�>���Q�dI���T�+�|}}������k�w��Z���U��c������F��n��8x� ����c��	F��*[�,:u��A���G_����M�4��-[#�r���G���1j�(����i���5������?�����x�����5U����#F�[�n��};.^��r���r�6m��-*LLL0{�l>~~~����N����j������]�v
S$*R�vm8p���8y��� ���'��_d2Ct��U���������|����q����?�������vtt���7���[A��/]�H��O�b����~�:bbbT����u��A�&M��7����^��U�T���;�i�&9rD. ����]�v�w�}����8�&&&;v,����K����iee�:u��c��h���Fs:�sf�����A�^������@���!���0a��d����F�V�p���<yO�>UZ�V�bE4h�m��A�z����
��������-���Kh��]��SW����s����S.��M$�y��5j�Z�TY9��YS���j���uC��u�q�F�?^���2e��_�~���_��Fuss��}��c�:tH�����9:w��#F(������-4U�V-����z�*����{�����e�����������C�����d����u+�o��}��I�U�ejj
ooo�=:�l�[��������o�>���S��khh���[c���2@$/,,,p��q\�r7n������{�D"����������K��|���puu�����K��Z�*�����&�;a`���P9�������C�N����;b��o�g��APP^�z���8����Z���\��n���m���]����7���#v�)
8��W�����

���WFOO�������t��Q'��G�
��zF��JOR�s}�RRR���C|�����HJJ���),,,P�\9899}Q�m�D�'O����7���FJJ
���agg�Z�j���E�����������@BB���Q�dIT�^��W~5!�H���3��XZZ�b��pvv�s0[,����x���t�kv
���K���	e��-R<BCC���K|����������������������ln���X���?FLL

akk�
*�f������?~\��:w�,�����O

BDD���`kk��e��N�:yu������ ���">>�������V�����%**
�_�Fxx8bbb���###����l��pqq��}1,,�?�^O���agg�/���%x��=z���(������*T@�Z��ut��d�M���8���*T���s�������x��9bcc+M�nkk'''8::��������w�",,QQQ022�������UF�������W����h$''K�
qpp������]�/^��������8�wU�D	899�|��ZwHNMM�����k���A__����R�
j���/���|$''���g

�>����666pvv��c�*�����@|��qqq���@�
P�v�|�X�=�kv=<55fff���D������T�����t��Y����Ai@�8y��5�<y�>��k��S��v��I;v������c�sJIIAPP��y���x�D"�(QU�V�:��*b��?��a$	���Q�re����)P�L~���7o� 44qqqHJJ���>�������n�`FF���"��fmm�J�*�����}_\e�������� 33���ptt���;LMM��Z

��w����{i���m�...y��{������HJJ���	����0K���������x��=bbb���333������	U�T���p~KJJBPP>~����h�D"����L�2pww/v����x��-����������	�d�KHH������U�~}�_�^��'0�NDDD���������+W���]�d����g����T����W����ty��!�����DDDDT\�����?��.���(�������#""""""""�/��a����$	v��Q�%*r�o755��!C
�4DDDT\���c�����
0x��@'"""""""""���������g��EXXX!��h�@���o�������
�����s����a����\�/�DDDDDDDDD�sC������y�}||
�D�/00`ff���ri����8HII��M����;w���[!����:����	&O�,]>s��>}Zx*��=���\�r������H-{������4iR!���gP� """"""""�/���
�DDDD���a�0l���.�W�#�����������������:@O"�H
�DDDDDDDDDDDDDDD��#�����������������:����������������0�NDDDDDDDDDDDDDD�t"""""""""""""""�
������>~��R�J�����]""""""""""""""�0�N:���GDDDv1��


��]��KBDDD�uc��������NFD�S��t"""""""""""""""L�NDD_	��.��2"""���u2""�@'"��B���DDDD������������c
w"""""""""""""""0�NDD_�7o����7�]"""���eDDDD��u2""�@'"��BLLbbb
�DDDD_=������
�dDD�1�NDDDDDDDDDDDDDD����������������0�NDDDDDDDDDDDDDD�t"""""""""""""""�Aa���� �.]���@DDDD`������(`���H9��������P�E """"�^FDDDT�N����4$$$ 11iii���,�"i"�&&&������9���
�Ha�������4���������W�
�,Y�k�F�n������{��y�O���{�	�H{{{8;;�A����#���T�Grr2�?�K�.������������\-Z�@�F��������q��	���O���������P�~}t��	���:�<�D���HDFFvQ��������"66���ptt��~��Y\O"�H
��e�[�."""����;w�vq���>}
�V�Z!������5j�����v�:u����?��...������_�����K��<  ������t��U�b��������4,]�G���G�����U�Vy�����2"""���:�z���+X����HTH%"*�b��y���+V,6#�9����
)))�]""�/��O�%K����<<<������L���v����q��Iddd`��J�+!!&L�����[�F��mQ�|y���������U������:u*RSSaii�^�z�n�����EJJ
^�~
DEE������c����v��~��������������'��.\��~�d��>��r���"�������
����&HTX233��������b$''#11�]4�p:�G�QQ�]�v!�������<y2:u��6m�(y����o��l�����
���_��S�`dd�E��e��
��H$��00�����{#66U�V��5k`oo��}���s����6m�===��9�{�V�mFF���~�X/#"""*|���.""����R�J)}� "�������A�2e
�D�)>���������HX�j����imll0e�����������S����S<��$*F�]����011��+T6l�%x�m��z���2x��s""""�b,))I���+Q1faa!�����\��Q�DDDDDD�s�����?44T�6��7����}�j����8�9s��C�|��~��addd@__�����Q����:�r�s"����K���s�8`wx""""""����4������n������+��
���@���O����	{{{�ze������
��)))���LMMaoo��9
��?�^�:J�* +��������{{{����y?DDDDDDDTx@'"��B���DDD_���@��+U�$��g��I����HHH���/N�8���xY���������N0�=��J�����G�a��u�}�6233���������#���0::Z:o[�*U�����[��������3���nnn4h��i��~���>�����c
w""�*���p�"""����)�+����m^�z%�~��!��g�4xd�R�u���'x��^�|)�@@���;��yS<�����������3�>S�����`�����a�4x��9������/�j?_������
�dDD�1�NDDDDDD:�{�n<z���uk��QCn���X��w����o��I�&��};�]��s��a��Y����D"���kq��e���������E�����q�p��	\�~���C�.]�?���������g���c�����puu����^��K�.a���(Y�$�����w���!""""""���:}���T�� ""����sk��������~R�]rr������h��!V�\	WWW����{���U��s����_�H$*�g���1b`hh���+c��9��� <<��s���J�*���E�z�`bbKKKx{{���W:����������t��>�����c�������t����1c�b1����x�b���)����X�<i�$�D"��<<<��ukYi�CBB722������:uR��	&H�=w���J�~`���011������{��5��������������\]�tA�z�0w���.�Z���Q�^=��W��/����.aaa�8q"��� ��`�xzz*����L�[[[T�^]���7���=�����o�����7j�H�{����F�

�������CCC�>��~�����A��6j�~~~���`�l�R��}�0v�X�366���-�U��o��^^^00`����,""""""��O�>a���������0{�l�j�J�kJ�.-��R���6&&F�o<��N��dff"66%J�P���2XZZ
������V{DDDDDT|���W�E�W�9��7�G������@DD���_������F�u���������;����?@'""""""����`��	��1�;w��uU�T��?33S��b�X��4��+WV��"9���HGGG ##C��(JKODDDDDD����[:=$''#88�v�Bxx8���1m�4l��
zz_v�*x��������������'������'�o��j��L�2ppp�5��D�|4Chh���%K��-g��� ~n�cll+++������������D��������z""""""�gkkggg��Z�j�_�~��s'*T� k�/�B.)}�@'"��B�j�P�Z��.�#%%?���<y����0l�0���M�6���n�R���K��������N�:������+�g��j��
}}�����/_V��u���x?_:������
�dT\YYY	�=�_�^x��/S��W������@DD��HOO�������o1~�x��g��8x� RSS�r�Jl��	�mN�:�;w��5k&��M$a��AX�f
���M�6a��1�m222�d�ij��={��%<<]�v�5�}��
r�t��[�lATT|}}��iS��w����S�d���]��&_�W��2"""���:g�������@V�w��y�?FXXRRR`ii���+�y����������o����.�8q'N�l��91::�������;������������D���������~�;w����7	KKK���c����U�V����w��o�>��u���@�%P�N���5k�T�z�X�C����x��5�|�����F�~�r����t""�*������.�����q��~�����BBB�nohh��+��wpp��1c�z�j���`���:t(\\\�����/����sssL�:U����������'O�q�F�y��;w���BCC�{�n����iS�m�V��mff������_~��O�0d�6nnnHOO��k��k�.��b�D"���O��O������
�dT��_x3���?� �g���A`` q������prr�I9N�>��"99Y�>>>�=��G�p��y?~\�{\�t	�g�������/������;��k���~~~�������!X���0�<y#F����c�>))	?�����+X���S<}�g������?���R1�NDD_��O�G��@���o�F���Un_�L�
C�A\\�o��7o�`��yr����a���pttT����X�j�L��������?��#�]��M�p��<���k����\��>}��e���133��y����S������
�dT����]�D	Y�������E��YS������t���?���0L�6
�w������=������dL�4	�>}B��-1n�8�>MMM�'O���9sd=�v��M�4���=������s\�r���S�9��;�%J`����Y�&$	�_�����#55,@����S���c��^�����^�z������x��
�������c��M���Q��>{�li�������#>��'N����X�p����:��'�E�8p�������H���-Z�@����R��*Q��n���G�����x�����amm
WWWt���[��Iy�����u�b����y�&>~��H�r���q��0`����������t'##�v��.��[@V\Q�k777|��7���&M��7o���������t�r���������pvvVZ���H,Z�@V��u���m��>=""B��<y�5j��������j�B�
0{�l$&&���S8p���/_���u��F�������k���v��a��98}�4��[��;���J����W������p�b�
���f��a������U��T��Ft�����pww��{�W�^������-[��F��J�*�5k���!""""""�%''#886l��de;���o@i��l
6D�-p��e����������oi����Ye����A�{����
;�{{{c���������'@��s'222P�fM��y6}}}�����p����p�����C������������O<�6b��?/^�P�9�4�Q��q�Fl��Q�����222R����h���#--M�.;
��g��T6Y#�[�l���8;;���E�����P�Z5|��	aaaJ���M���e�������t�X�;w���\P�dI�����G������j���;����������������+W���i������N��{��������[���=bbb��FF�t4������un���T�=;�zbb�`����
X�v-��]���>�,�hh�t���������/�DDDDDDDDDDDDDTd����{��5���666
��������9���j]���XH$@�%�~011Q�w}}}@ff�`}TT�V���@\\����#������5`����
�k�.�"X/#"""*
X'�����V���9��uK</W�888���D:����6m��oe.(9��F�B��m�z������yE��b�������������������Jy�m�6�#�U�uW���5����������<���e�&���h���������h�v�{q�_� ""*111y�������t��2"""���:}I^�|	�[���t��?V�>���600@�*Ud������ �+WN������Z�G���allx����m�����Gq�:}��y�7o�v1�����z��>���K"�������<|�P�����Tn��ys@XX���_M��"�M�6��q�^���=P�n]���7��������'�/l1�:K*T�5"���wr�����������(Q�r��}�J�_�p!BBB�n����\������C$!333g�T��X���O�m��woY,X ������[U~�/�@'""""""""""""�b�S�N���Grr2F���C��F��R�����?����������@pp0�m��&M�H����(U���@��Y�0g�DEEa�������4i{{{$''#$$W�\��7op��Q�fggg������?���K���=z�@���agg���4�������p�"##�w�^�.]Z�-Z�@�������1���#���p��q�;w5k�����3�NDDDDDDDDDDDDD�����t��������OX�|���"�S�NE\\��z���q��A���b���X�v��o�����a�t�S�N�H$X�hRSS�w�^���W�=��)��O���`jj�+V !!~~~���S�����t����������AAAx��!~��g���U�����
���PT1�NDDDDDDDDDDDDD���9sP�~}>|��=Czz:���Q�N���nnn���U��J������u�V����HMMU�}�����Q#�����_Ghh(���`nn��+�^�z�����?�@�=��E:t7n���7o#������iU��� 98
�X(h�i�*V����to��������;�}��dV��v;XH��P�P��)��y��A�a8��7���Ak�y����Z�Y����F���?�Z�sv�m����>9��C��| �������{��;67�xcn���<������H���s��G���O��y�����������C�c���ICCC���S[[[t�����%y�s���x�d����f�����S��w����vk{���t���h���gNPZ��@{�t�BCCC������$lO*.�(:�pjz����o6/(��2����@Y�3gN���St��7���2��Y+(M��I� �:$I:������#�d����f3(M��0`���#��������|�4�p�(�P&���S___t��7���2��Y+(M����������c��A;2/(��2����x���r���g��)y�������N�:e��v�A��O>9|���0aB.���6]��K.�I'��I����|���n�?���2z��M����S���n���>�����g���w�}s��'��c��"��t���w�yy��G��|���y����/d��	9����/});���6��d��\y��[���c��g?�YV�Z����y�2o��<��C������o|#]�t�*�XM�`s��M����n1bD>��TWWg��Uy���r�u����_�-���+V��+��h��\sMv�u�������I�����^J���3��M:wCn�������O�d�=���g��}��'s����q�2i���w�}��W���qlt�����^3fL�:��t��q�cx`N8���s�9y������%��Gr�!�l�������c�=�H���~:�_}:w��O~�������H�,�~��$Iuuu����TVV�>|x.����{����_����2d��6��t(:l�����;l��&-�d^�W_}uF��N�|����|�3�i����;�U��\�2�_~yV�\���>;{�������n��E��$�����*�'I����/|���������6�3ke�)�P���S]]]t��W�����MZ���_|q�]����m�������5�����E������$��w�QG��6��������$�#�<��^{m�f�rd��4t����e�Z�;t�6K:�g����c�$_|q:w����^�|y�|��$�;������N%��~\��e����Oo��^�;6C����v@��PWW�����c��Q�F���I�'On��{��7����.�������=��>���:+������_~������+������;.C�}S�K������+����
Y��s�=�Es@9�VF{4i�����/qn���^�z�����rB�E�������\t�T�Ten�&�Z�*�^{m����#7zNmmm����`��<�����������G>���o���<������Z�_�R�,����w�m�|G��9s�x(7�c[^EE�	���������������m}���������~%����?GuT<������Y�r��w��;����K���=9��S����r�UW%I�����{o�%�/n�������y|���4t�v���6?���$�{���_\�������x���x�-P�x�;r��G��{���>���X�"W]uU�<��������s��Wg���9��J�7���K[�7���7_�<�-�;�������s��\V�\�.]���+�����=z�X�x��������x�����?�q�6�&M��	��c�\|����a�,u���u{���l���5��������5+\pA.\��;��+��!�����r�);vlZZZ2y���{�����-[��}�kI��N;-��������;�n/Y�d�m�<��y�w�uW���?��'�Hccc:w�����->|x>����W�^%�}���s���f��)�������K����7�rHN;��8p�����k��6<�@����^�ze�����>��z��[r�l&t�����@���&��W���;wn>��Of�������������}��"}���;���K���2w�����y��y����S�80��_�9����k��>}zk�8 ���os��}��n����l;g������u�<ke����?��?��#�����e����O�����
7���|�;9���9��	������|��y�����s����n�E]�SO=�d�G}4^xa^{����^y���{�����{�o��o�1J�4t�Beee�H2c�s��hjj��1c2k��$��>���x��[�����Q�+V����_��~�����y��I�K.�d�
�H���r��<���l�����������|�r�l��|����3�<��;��c��������g���<yr������??����s�u�e��w_���+W�W�^y�{��w��]y�[���]���W^�3�<�q�����)���7��^{e�����hhhh-�w��!�����| =z����>�_�����O~����[���
����E�r�d��I�.� �F����hllLSSS�d�]w��}o��v�)�x�;��c����������N;�������$�;w�~���-cP����L�6m��^}�������'�<�Lz�������^gy��������g��W^y%?�����a��c�=6]�v]��#�8"���r�y���g����c�[@��w��z��W���{��������1"��w^�z�����mC��0u��$�ATp��6z���:u�y�z477����t�y��$�9�����:k�_��CZZZ�d�w��t�I9���6x��I�r���'I�;���=�Mgy�{���{,���Z����s�1���3gN~��$���C��{�7}=�u��h�n����p�
o��������}�����/�%��w�=���s��W���o����t�������Z�=zd�������2e��455����W^y%w�uW�d���k�W���{����k���ys:��-_�<]tQ�b�����O~��������[����{����O�$���K>�����m�3d��2��;?���G�I�k�������V�\�+��2+W�L��y��[-/;���'g��EI�|�l��w�+���1z���7�v��%�={v�O��i��e��i�����_��g�]���I�Z�������n��l;�@(������J��]�'�|�Y��N;e��k}6{�����y�;�����g�}�M����$�f��w��;������/�p�w�lm���K>��O��_�z^z���u�Y9��s��>�d������~�I�&%I�9��2���lm}����^��o�y���|����.e��y�|�������.w�qGf���:�^�7~!t�����{��;����W9Q,t����ok�~��G���}l��w�}�L�0a��V�S���]��?��?r�)����[�G>�����+��O�_|1_��W�is����+_�J��566�����������s�d��m:���k��y��/����O�6�dkS@����~���������O?�W^y%MMMY�rez����f������>����mm���y�{������y��G3�����3���oN:��M�kV?6=I~�������
���_�������/|!,H�N�r�i����}o�����W�^���s���_��>��$����ob$A�`�S�9�w����;.�w�HT��!C��w�=���qt�A9���6'$y�UA�UUU�Uo�Gy$�f�J�|�_h-�����K���W���y������d��=>�bt(:l�������P��7�7/(��2vtk��=u��7����#9rd�vk�o����g�6�k�q�t�B��]��k��c�����e�V����Cm�o|��q|�z)+V�h�^�d�z��Z�*7�tS�>���;&In������|��L�>}�3�u(�P������\t��W�����`�������3�F�J�<��c�����j��������N!��o}k�v���5�\�g�y�d����k�<��$�=����'��f�������V��=�@�,���%�w*lT��������Z�����������O�w��]jkk��}(o{����[������>}z~��<���g�}�z���a���w���???���2{������Oeeef����n�)?�p:��
>&�3��L~�����k�������'��@�w��g�}6�^{m^x��������N(�;���;��?�a.�����o�?���|���,��{��k�w��-�]vY.���,]�4����?�a�6555���?��N;�d�{��G��������l^{�������������w�yI���P@v8��w�����L�2%7�|s}�����+innN�=�������G����=��?l����W����^�I�&���1={���{����;.'�|r6�c��!����s��������+���^�ze����i���a��e���[���7��������jjj����������`-����Tl���*��;��{��'�|���@��6��g���+��S�����E������w�C��=P@�x:e������/�����q�1���2����Q@�L455�����eo`���e�VP�G�P����$����(s#��L}}�y@�����t�;��m�G?J��-:�xZZ�N�?w�@� �G�o��3g��{��C=���~:�g��k���=zd��w�!����6lX������&]�c��y����~��Y7n\&N��Y�fe������_���w��SN�������M������k�H���1i47(��2��*ZZ�iS=������G}�M��
��}�{-�o���~��|�+_����K�>|x�����o����c����444���:���o�hO*.�S�*.�[�U>�rf����K����"�v"�	---���KKKK:w��A��M���&L�>}���������=UUUY�pa&M���^z)I������`���?d��m��Yg���6;vlS_����E]����.����O��=SWW�)S�$I�����q����������o�M����t�����d����&�&��;���wj{�����k��q�9��S�����ul��U?~|���/e��%ihh�\�?��Om���W\�E2N�6-_��Z�O9��|����;����}�����G���)O?�t���/�����"�h/�$���'(o5�j��������Z�����+���j�d����y�����h��������P:`{��o�\u�U���{2f��u��I��C�|�c�~����&O�����{[F�����,_�<I2t��|�{�[�x�$Gq�Z9o���<����4'��6g����3��eoH�!2���H��6�G���E���_�k��oF��[�re����y���~��G�mw��	��
��a�������������G��q�y�����������[o��u��_�b:tX�w&�:��><��{oV�\�_�����W��Mr�':tH���3k�����d��EY�hQ***���](G---Y�r�Z����n�����Z@�9s�6��_����Z�*���h:t���5*��{o����nS@�V��=�*�'�W�XQp2�~���.���O�16��6��;1�������h�n���vX���Y���s�e����*����={�mo{[-Z��f��e������;f��wNeee�v�Zt�M���
<��3��{��G��y���2e����;7;vL�������g��!�����5�c~��n�}uuuv�m���;��|t��u��!�z�J�^���l
�[��Y�r�����>�M�}�#Y����u��>��\x���u�]7�����[���s�6]�����5���UVV�$�OO�W��Y+(M}+���K[�����9r��[�dI~��_��[o���������d�������v��M�������Me\�jUf����6m�`K0`@�Hr��������d��4��h������[[�/���t���d�.]������QG��:(���O�.]�����S�������9s��_��_��?�)�
Z��������������^k�9o���/g���l����jtP��������6x��$ICCC����N�����_������@�~�R]]�$���[���
0��xS�N]���]����9uuu�mcL�dL�dL�dL�dL������SUFU�Zo��/�M��60#���e����?g|���WM���N�����^PL2����y������R��6I2���T�T�N���&f��I��{�^o��
���������1�^��������8�)#G��3���k����eMMC�Y�g���������#F�g���u�4�_jk_�1jT]����1q����Q�����5I��������j��Q���gL������W���;wG�G�1�1�1�1���i�g���V2u��\|���������x��L�4)�{�^���v�-#F���#2q�����innNSSS����+�_�:���?��v��M�;w�\���������+W�v-@����g���y���EG([���j���c��������;�^x!'�|r^~��$�~�������M��[fq������?����?���y�;��V�y������w��}��g�}����0aB��������_��\555ihhH��}s�-�l��G����o���5�O�Uv8���<����~�-�*me��4w�oas�������Z<0`@����-V<O��O?=����2k��$��w��N}��w^k��w����{��o*_����N���H�����O?=�?�|��}2n��-��:���o��6m�:m�u������������l��}�@}y��W�O��O�/����w�������[�z}��m��?�z�4�u��_lS���j��;:�-`���9��3��c�%Iz������.o{����5�,Y������������n?������a�;��<`G�������s�Yg��GI������_��N�-m��x�G�v�a��>��F�|���Z��������{3�/�������`�MZ8)�&���Z@i
��a���9���r���'I�t������:t�V���i�2i����a������G�^�W<}�����n������ns�1[ )@�Q]]�����c�������5/(��2������+Wf��1���;�$�:u����y��o���^{�M��,Y�/�0+W�L�����������n�����;�u��+�HKK�z��}�������$;v��g��)��{
�oBKKK>�����[nI�t��!����s��G��>=��|�[���i�J�y��Gr�I'��Gm��s��\�w�^������i���$�����O:�/^������1c��������o(B]]]�����P�F��Q����d���NE����������������?�������u>kll��W_����:����o�������K�.ijj���?������9����/��/��>�����������$7�xc����v�a���g����U��o�����~�M���477�$U;U%U�fE�VP���0o�����{��<��sm>}�5544���������|��_�g�����~��iii�%�\������)��z�:��8��|���O�=��/��D�����{S[[����<��S�7o^��k��{�����O���wf���9�����[�M���3��{�����w����3k��,^�8}�����SN9%�{l***����������C�c���ICCC���S[[[t��L�:5Ir�A���I�e�`
[��=G'�|"c�����f����VPZ��@{�����$�ML&^Yt��f��4t�Beee�H2c��dFe�1���2��<���@��:uj�N�Zt��7z��=���H��JS@�(�@tH��I� I����-<��$�0>u�1���2���(]�v-:IW4&��fE�VP�G�P������\t��W��*UU�eE�VP�:e���.uuuE�({��Ge�(�2�"Y+(M��I� �:$Q@�$IEKKKK�!�1������!������-:l���*��;�K-E��`�6�;� 
������455��
�606��Y+(M��P__�����c���}Ff�H�2�"Y+(M��I� �:$I:���]��$���Fs3�"Y+(M��0x���#�d��������P������t�DCCC��P�jz������H��JS@�,��3's��):@��kH�1/(��2��� 
��D�(�@��S�`[���,:I�/��L�,:@Y�VP�:ea��EG ���oOn77(��2��<���@����O}}}�1����#2b�y@������@YhjjJSSS�1����e����c�5ke�)�@� �:$Q@�$I������_��#�d��I�$s3�"Y+(M��P]]]t��.�Mj���d��4�p�(�P&���RWWWt��7����e^P$ke�y�;e�����$���*�27(��2����Q@�$
��D�$�������@���&&�,:@Y�VP�:e�����$��dF2���e�Z@i�Q@�LL�:5S�N-:@�����m^P$ke�)�@� �:$Q@�$
��$�Tt�\t��o�����e�Z@i
����]��$�+�Fs3�"Y+(�#�(���inn.:@���T��*�2�"Y+(M��PWW�����c��Q��2j�y@������Q@�$
��D�(�@���������jjj����������6[�eEG����"`k�����t�DSSS�����P�v�����P������@Y���O}}}�1���>#3r�y@������Q@�$
��D�$���B��]��@����I��@������@Y<xp�H2~��d��E�(k��J�w�:e���!


E�({5�jRSc^P$ke�)�P����9s���
�5$C����Z@i
�tH��I� I����-TVV�$�OO�W��Y+(M��0`���#�����'����Z@i�Q@�L��������eoD�1���H��JS@�,455�����eo���2hPS�1���2��� 
��D�(�@��S�`[���_�H2i��d��@������@Y���.:Ij�&��fE�VP�G�@�(uuu���+:@��oTF�2/(��2��<������\t�T�T�T���Z@i�@�(�@tH��I�NE�ma��EG ��y��W��Y+(M��PYYYt��X2#�QYt��f��4�p�(�P&�N���S�����stF�6/(��2��� 
��D�(�@tH�t*:l�.:I�7�O�_]t��f��4t�B��]��@���I��@�����������477��Uu�JU�y@������@Y���K]]]�1����Q5���H��JS@�(�@tH��I� IR����Rtv555ihhHuuujkk��������#���RK�5X����t�:e���)MMME�({�
���ME�(k��JS@�,��������eod��9���H��JS@�(�@tH��I�NE�m�k��EG I�������H��JS@�,<��$?g|2���c�5ke�y�;D�2��������c���^5��1/(��2���(s����9s��P����!C���d��4t�:$Q@�$
��$�Tt�*++��@����'�+��P������@Y0`@�Hr��������d��4�p�(�P&���S___t��7����a^P$ke�)�P������Tt��7h�A4���e�Z@i
�tH��I� I����-�����$��pR2���H��JS@�,TWW�$�k�Zs3�"Y+(�#� 
�������������7*�F���Z@i�@Yhnn.:I�v�J����d��4w�@� �:$Q@�$I����0`���#�d�����+��P�������f���{��'=�P�~����=;���Zz����w�=�rH>��g��a����)Sr������K/��$�}��3l���v�i9���7��Y�fe��q�8qbf������_�~y�;��SN9%�{l***6�_��Aeee�H2c��dFe�1���2��*ZZZZ���y��'��/|!�>�h��6,���������]�lY.������?O�59��s��/});��S�2��w��W���,^��d��������~�����>����&


���Nmm�������|���K-E��`�6�;�������S<8p`�������*.��I�Z�����~0��6�X��}�s���Z��C9$I2y��������%?��O�h��|�;��h�q�����.j��e�]r����g�������)S�$��{o�8�����L�����g���:uj����*8	@y���d���k^Pke�)�o����+g�qFN9������k[�jU���/}�KY�dIr��O�S����7��x��C�|�+_�����:������,_��W�j���7.�y�{r�����8m��|�_h�?��S��o|#;��s�g��w_F�����<������������M��l�:`{��o�\u�U���{2f��u������}�c��~������s��w����K��u7�'?���w�y����}�w�y��'>�������,[��d�o}�[Y�|y�d��������V�<I�8���r�p�
y��gK�	�#R@�
��N;-;v�h���;.�z��Z���������_����g'Iz���/��d����g��g�$��/�X���s���[om�����VA~MGuT��$Y�re~��_��H}2dH����3���/�K��I'��n�����[�n9���Z����?���_����Z�*���h:t�s�5�u���n�`[���6��;�W�\��6<�@��a���>�ls���o��a��mR��f��s�=��sv��P�y����=��c��.��9sZ�8����y���n744��W_m}��jk��|���TWWg��v���s[��{��7z��`���EG ���������P�������l��Yk�!��=�k�>}�Z�����h�ol��>����{���>������^u��5]�v-:@�k\���F�2�"Y+(��[���^�������g�����illl�������Z�n���G�,Z�(I������%K�����u��vkS�5�����Z�jUf����6��`kZ�3����SUR���P ke�)�oE������������K�.��{���Z�7��U��][�k��$�/^�m[�\_�M���/g���l����jtP�����������2


k=�~����0 IR__��/���/���I������h���RYY�$�:u�z�0x��t��5������[oc2&cj?c�7o^:w��#�8b�S���{2&c2&cjocJ^/�����6c_�$�m`F�Y��e�7f���I��^5�k�:m�/������$�{D�<h�6�NJ���$��~�R�S�:m&���Kf$IF�9z�y�7�O��Fc2�B�t��K�K�SO�Y{Lc_�1p`SF�\�gDcc���������������+s�����1�>�5�;�I�R[����Q��RU�����d�����4z�?#��������j��Q���gL������W���;wG�G�1�1��1��7/I�����0cZmG��dL�T�cZ�����[���Ss�����C��?����]�ti��N;���kt���u�����q���n�O�YEKKKK�!v4/��BN>������I����/������={����	r���'y��S�Li�u:�����+I����'9��Z���7/�|�;[��������>�������	�$g�}v.���6eI����444�o�����[6��#��mm�7��������*��;��{�N�D�]���c����VP�G�oas�������Z<0`@�������$���{������f�5�H��w��d�7�g[u��A���x��4���~��y������^�q���_�~<�����������b��%KZ������u��m�����;�
#X�����)�o!���j���������{���q�����u��4h��_|q����5k�}�������~��'���#�������3��c�=�$���W������mok���z�J�~�2g��$��O>����?���������}D�����'�|2I��Ol4GCC�Zw�����m��=�>'��a��c��?.:@Y�VP�;�7Ssss�:��<��#I^t��~��������~;����x`��|�����?|�}�����z�u����{��7z��B}3,_�<��w^����$I�.]����<C�����9����	&d��%%�.Y�$7�|s������vG}t:tx�_����S[[�����_o�ASSS�����P�v�����P�������&�\�2c����w��$���S~����#�|S�}���}���$,������m����,X� I���{f���m��n�����k����+������w�}w����$I��s��g��q�W������/:@��gdF�4/(��2��������|����-���$���C��������~�}v��%]tQ��5�\����gY�jU�g�V���~�����w�g]tQ:w�\������i���$�����O:�/^������1c�����m���v���=��/���������?��~�M�_q����c�Xx���x��Y�jU������?�y9��$����������?���r���n�Z���O������o����q�9�����g�������>��~����W���6�`G���&��7o����{.�=�\��/U@O�o�����W�����������_�h�$9��s��/�M�;�������K.�$�/NSSSn���u�q������=z�y,;
�v�s���������|$�����>����$Iuuu�
��O?=|�&�{�g���}o~���e����5kV/^��}����)���c�=6[aT�_EKKKK�!�1������!������-:�Z����$�.8	����|���Q�F%�j��7/�-�*me��4w�P��@�0~��d��E�(k��J�Pth�(


ihh(:@���U���2�"Y+(M��0g����3��eoH�!2���H��JS@�(�@tH�t*:������'�|2O>�d�z��<���Y�`A���O���^���f���n�!?�p^|��,Y�$��w�^{��a���#�Hz��������|���n�?���2z��M�������o�1>�`������k���[��s�:4��zj��c����G��PYYYt�L_<=�^Yt�!}��[��[n�%_����t���>_�pa{��<��c7n\������=�is�K�,��W^�E2�������e���k}�h��<��3y��g2~�����WN8��-~���2���((:In�{r��li����k����Cm�yS�L�e�]�U�V�C�9������7���[r��7��{������~6�_}��s�6����(/��Rz��������a���_�g?���b�{�����������ss��w���o���Ks�e����9������w$��JS@���y������?���������g����&�q���f��UI��}�s9��S[�����>��|����u�]��K��������n����~:�_}:w��O~������7mp%���$g�y�Z��O����}<xp���og��U���ks��Wo�kP:������������="#F����0z��><}��y�}<��cI�]v�e�����;�����|�}�\�2�_~yV�\���>��w�����9��s�����N�.�������Z@i
������455��
�yP
j*:��/_�$����6=z�h}g�������6uuuy�[�����"9W[}�]v�%=z�Xo�������%o9�VP�:������5kV�6�-j-�n����g����c�$_|q:w��e�����_�`A-ZT����x�7�J��G>��$��o��������~�N�R����477�������C�\�7\���%���/��f���Y�`�Z���:(�?��L�2%��rK���o��g���G�]w�5


���[s�]w%I�9��z��%��������g����g>�U���=��9���������/�^x!�w\�����+�������7��$9��s�I'm�����Lu��1�]vY��_������r�M7��f��!9���7X<_�`A����$��1c��w�������djjj��_�"�������m����9���|�[-;.�p�,���/���+:@���pR&M2/�������-���i�������?�?���y���K�q��Wg���9��r�)�l��I��s�f��	y����{|��i����3c����c{f��4w�P�����@����I����>�h>���d��E�}����O|"�zhv�e���7/��sO~�����k}��\s�54h�Z}L�4)&LH��s���C��w��s�=�1c����_�.����|�3>|x���������~���{���<yr�������l�<�+ke�����e���/~1�-J�>}��_�"�|����N�:�_�~9��S����$]�t���ss����������$�i�����o���\rI^~��t��5?��O���~4���K�N���O��p�	���k��O����k�����e��m�L�X��@Y���K���?ql��~��Qu?������>����N������v�
�q���n�)O?�t����mo{[���;��/��N�:e�����_�����=�\�����[�p����������SO=�$9������{�������Q�F�G?�Q����x �{���|�r`��4t�Bsss�HR�SURen����?������}�m�<�������/_�$Y�bE.����^��;���w������7���f!~c�����o����Z@i�e�c����+W��`�+V���m�����������P���c���G}4��/�v�����k�5~�I'���N��u&M����??Ir�y�e���o*����2eJN;��6�]s��1�@�2��w�;]�vM��x���6m�z���������$}��m}|��6{��2$C�������s|������o��������g�y&7�xc��k��:t�V�������2eJf���������=s��L�0a��o�S�g��9������8���Z�9���v�i9��C��g���??w�}w���7�V�J�\p�����{�:t��1c���K.���+�����9��Sr��G���*.�C=�������g�uVz��YH^�O
����$�ML&^Yt�!�t�M�����{l����:u�Z���Q���{n,X�q��e�����/~�_�������S�����?~���N8����???���b���?>���_�]EEEN?���{���l������@Y���,:If,����,:�***���~6�|n���L�2%


innN�n�����%�rHN9��vSt=��33|������o&O���3gf��%���k���s���C�P��o����[��J�hiii):;����444���:���E���VqYE�`�t�������+�E%�����el{��������d��4t�:$Q@�$
��D�$���������@��
���W��Y+(M����k��#��qEc�hnP$ke�y�;e���9���E�({U��RUe^P$ke�)�P���RWWWt��7�zTF�2/(��2��� 
��D�$��RqYE�`�2z�����O�]r:$�hiii):;����444���:���E����N'�J.�[�U>vd�e���\������@�hjjJSSS�1���n3p`S�1���n�����@Y���O}}}�1���>#3r�y@�F�i��t�:$I:�������'���O>���z*O>�d,X�$9��s���n����g����&]s��w��	������k�1k��466f��E�y�����������p��k�M���Z�*S�L��>��S����>,H�.]��_�r�!��G>�}��w���t�?G}�6�����l���������W_�3�<�g�y&�_}�?���u�Yo��'�tR�����3f���3��?�!�����O}�S���x����N����k��#��qyc�hnP����������:{��Wz��6���o��7n�������v�mI^��}}z������p����v�u�t��5s��Mmmm���?e��E���k��G�|��ms�5���+I����-9���r�Ae��v���K3i�����������_�*;v��1c��u`{P����Rtv555ihhHuuujkk�����2�����RK�5X�cGf^[^�%~q�2v�����������O�����G�����+s��'f������{������K
+W�L��K�3k���y��Y�pa���r�m�m�})��sN�;����=�Y���/��b�>��466�c��������{n�u`{����^�=:��O�>}��5~����;7Ir�QG�|"���������#�$���y����T�����6lX�G��������?�����]w�����t�BCCC��P�jz������H5�j����[ni�.�������{���e�6��
2dH���Y���u�h
���9s�d��9E�({Cz
��!�eE�k������k�wp���9��C�t_��������$:t�[���-q��/_�����#;�NE�rq�w���9Ir����|lz)+V��+����S�����e^x��$�?����F��jkk[���{��v(�:l#k>���Nh�9�g��?�����
�/�ps���������wI���;���}�V�M�����L�<9I��w�3oy�[6��������g�:��t��qKD\����ihhH��z���m������h
��
�z��iiiI������o��7n\�d����;wnx��������������9����J�?���?~|�����O|b�\�t�Beee�H2}��dze�1���������[oM��c��>��6���S����>����G���9������0/��B.���-�w��I����%Iv�e�|���L��]��5���Pt�����>���~�y@�n����<��y����$Gydz����}�����w�O�0!=��f���SO=��~��Y�lYv�y�|�{���{������J��[n��u{S��1�}�{[�����-�������O}*���Z:w��o��9���H���)�P���S___t��7����a^P��GX+��V�X�����I���{g��a[��������^zi��{��3f��,X� ;v�����������~a{��@YhjjJSSS�1����e����c��A;�V���w�}Y�`A���c�I�N��X�/��r�v�n�6��9s�����d^y��t��!�]vY����mfB��(��V����O<��-���������>���~����O~���={v����/������`{��[��r�}�%y��=x��6�w�]w��W^�`���'��?�i��c��9��c��n��!2dHN:�����Ws��>��?��?���M9aG�����)S�d�����k>�~����0a�Z�K�W��_�����'��������\|��9��#2t��80={��������/��{������U�V%I�;�����^m��e������t���$I�;��������i�J���[�������t�?7�tSn�����:uj�N���g+��~|{��7�����/�������o+��K�.��'>������W{��W��c�����������
�s�!��'?�����w
���~���$�NJ&��i��I����^x!O<�D���C������s����=�rH}��L�>=���Kccc***��.�d���2dHN<��M���������C�c���ICCC���S[[[t�l�UvL�Z����*;2�2��Z.��`}:�t�B]]]�����P�F��Q����4��(ke%x:e�����$���*�27(R�NU��Jp:D�(�@tH��@�0`@Pt��7q��L�h^P���&Z+(�S�`[���,:If,����,:@Y��d��2���Q@�LL�:5S�N-:@�����m^P��{��VP�:D�(�@��S���~��d���S������l.w�@��@�<xp�H2�a|2���c�5s2���(]�v-:IW4&��fE2'(�#�(���inn.:@���T��*�2�"�����@Y���K]]]�1����Q5���H�d�)�@� �:$Q@�$
��$�hiii):;����444���:���E���VqYE�`�t������#3/�����
s��_����j��<��3y���2s����;7�/N������m������%��������C7�����zSSS&L���o�=>�`�,Y����u��a��e��9���RYY�u���ijjJ�#
6���d`Sf��,:
@�2'(�]�'M��������_�����'I6����/��w��;��3�\rI�>���{��:t����v���>�:@�F��������EG([�d�Z@�����o|�y��G��[4���_��g�TWW�w�����[ZZZ����y�����!��M��/��z��e�r�-���[n���C�����9��C�����R@��r���f���I���y�~�r����������>}����y�����{������/ICCC���G�G?���92�^zi����n���+������?��-KKKK�u��O<1g�q��~�z�>}r��������W\�Gy$���os��7g��%��_��{��'��M��#`GQH}����e�]r�9���s�IUU�����:th������?�y~���g���[��X
)��3&c��I�^���u������~6��w^~��n�k��u����$i\��4���������/ ��TSS����TWW�����8��*.�(:��.�[�U>vd�e����VaN���Pth�(


ihh(:@���U���2�"�����@Y�3gN���St��7���b^P$s2��:��Z�ti����L�81s���������:*��sN�u�Vt<�3���>s���r�)I�#F��_��:m���3j��<�������1#?�p~�����n������2��kw�p������^z)


9������;��N&O��$iiiY��i��e��1�22;�vW@�4iR��[�n9����9��k���k�MEEE���3��/~��\~�����w����}����B��kw�p�6mZ***����=�;w^�����d��TTT���N�7����c�g��$�����#�8b���}���,:I�/��L�,:@Y3'(����$X��5�,?����:v��Gf��y��2u�����N��+l[���=����H�d���G�/^�8I��g�������$����;���u����~iii��/���B��iw��;&I�-[���y������KEEE=������O�$��+V���O}}}�1����#2b�y@���Jkw�����9s�:�������!C�����K�&Iv�i�-��VSSS�����P��<(�5�������
����oZZZ2y��,\�p�c&Lh�.u��w�����������%I������>K�,I���O�m��������-o�~������x��TTTd����F������Q���.�$In���p�9��2f�����$I��_�u��>��c���}�;��M��chw���{����J�������K������x~�!���s�]����[����wo����]=I�9������1��������N0`@���q���S�N���x���7.I��s�~���:6�X�~���_��c��I'e�$�2�"���V��������e����/'I:u�����������444���:���E���VqYE�`�t��������g^[�9l�d��uo��Nu��9{��g�1�N��G���VWW�����c��Q�Fe�(�2�"����������\t�T�T�T����������x����93���jV�X�������l�T�H�m}��E��O�?��O�6mZZZZ6��������[);�vY@��Gs���f���I���s�T�����K/��3�������Y���3`��TVV�c���`G��
��\sM^}��TTTd��v�����w�q���k����
0��$�8ob2���c�5s2���]����J����N���������Beee�H2c��dFe�1��9@i��F


�������x�6��
����K�d�=�(8	;��S�f���E�({��������dNPZ�+��~Gmccc�I('���~��'���%����b����P&�]}��Q���_����k����8��vW@�y������,��w�UW]��~���D`��Tt��9���s��7�����s�UW����uF�����g�����hS?��z�VN
���]���s��<xp{���������~�I�WTT(��j���EG ���������P���Jk�����;�������z�yKKK����u����$i\��4������vW@��gs�9�d��������?��.���N��]d����I��V��*�jN�[����������\sM�.]����4(����2t���c�����K�t�A'(o��G%��2v�y@Q��Jkw����?I��[��7.���'�t(:���7/9��#��f�]���*I��n���r��
�L���;��$��vW@?��������z(�/.:e�������kZ�tiN<��<��3����\y��EG��jjj����������6[�eEG����j)v�k��,�2�
��`�0'����;��t�����g�{��s�u���/��y�����S������n���@~��_��o��	��w�;�N�^�RQ��o~�3���Q��455%I*++�P�v�l���EG([�d����w���u
�K�.�}������o��R@`����$
�E�gd2�>c�V�l�����
�I����o����z�:$������EG�)�@�E������[C��]��@����I��@���JS@�,<��$?g|2���c�5s2���7���+SWW��S������Og���I�a����nhS_3g��{���M��^{���������}���7.w�}w^z��,]�4�������G?��>|���#)�����|%^xaz���M�7��\}����W��������r�d��%[����{��^������W{�����s���n��>��|��H�=
J	��444$I���NP�jz�$5
��5/(�9@i���������}�:��|��O�>}��u^y������O~��_e��E[���`���V<���G>���n�][�����o����n����_���w�K�.y���SWW�$��������_��W����	���9s�(�mH�!��9k
dNPZ!U�n����W_�5�\����'9��s���o�#�Ky���r�u���[o��e��������o���h��v�A��>8tP������g?��>+++s�Wl�|��{�Z��O|����?���;�~v�M7����l���s��w�?�A>���l��l/
)��s�=��W��	&d�������?�����oF���?<�zh�����������~8��&N���_~9I����$��?�/��[t���������k}����n��l�+���u���O�����u�|�C���s��'I~���_��_��#���B
����{~���������o|#��w_����_�u�]����.I��w�4(���{�����[�������9����K/��������q��W��<��|�s������->����4e��L�2%I��C�|��_,���3��O~��<��sY�hQn��������6J
P�B_t��w�+������?����'���[�t�����������7���b�j]�t�	'����;/x�V�������Z������k��������o~����*�����jx`~���k_�Zn����~�����������S _�^�z�����#r����g��� u����n6l��;����I�&e�������V���UVV�$�OO�W������.
�����3��vZN;�����d��iy���2s����;7K�,I�t��-}���[�������4hP***
N���X�"��sO�N������K�.���w:��|��m.jO�6�u�-w�p���+W���3��~�m����$���=����H�d�����***����f�}�-:�6�����O?}��*++�/��/�����{��}���+Y�`A���{����v��-}����y���^�W@�E���i������?�w\�O�^�]cc�Z����k������Z�z3V�Z���go��m���>���E�({#z����eE2'(����^�z�����?>�{��r�d��wO�N���+�d�����o~�{��7I2}����?�sn������g��^{�����v���k�{cm���/g���ls������A�������\]�v������~���9s�iSYY��������~	�_�~���N�������y�6h}_���S�;����k��innN]]�z��1S���y���s�jL�������������)I�:UeT�������$��n3���u�7.o��9��$5�j2���u�L_<���'��E�A;Z�����R��6I2���T�T�N���&f��I��{�^o��
���������1���I�I�3hP��c������2r��?#�f����^��4d��uFL�^����Q�#F��s�$�4�_jk_�1jT]����1q����Q���F��g�������5UU�5j�?�������V�J�����c��cZ�w����dL�?������4=��]��5&c�
c������y����5�v�o���<y�z���{d�=���'�����7���������������o�����t����;w���k�[�_�UEKKKK�!v$���wr�UW%I�
�n�a�_��+��~��$I��S[[��v�m�6S�L�	'���?}��6��~��'��GM�|��_������\555ihhH��}s�-�l��{���~����|+�kl�*.�(:�pF�9:y��w�[�U>vd�e�e����cN�������>���b���+s�=�����w���n�5���n�����C�]���ho��C��w�������g�}v�6UUk�{��W^iS�/��r������@�N����u{�����u�]��.�d��I�_|1�����lnn��y�Z�7�`{���M�3i��d��@���Js�vj�����;���z��Y��'6����?����c�8p3�/������.:@��]X��Z�2�"�������Z� ^� t�a��n?�������j�2dH�t��	�/
���{��'�g�n�6l�z�{������{�Z������[��9���L	����������eoT�Q5���H�d�)����-��e���v��y����[���w�x���m{�����N��\�2_���K�����&3f�H����#��zj�l������\t��W�SU�����dNP�z;0g��v�a����������m�����o�=�|���$IEEE���/�C������/�n�������,_�|�6���r�%����������f�`������3�<3


k}6w�����S�f�������_�z������K���+r�W�-oyK�����w�����S����G}t�k}�K_�>��
f>|x>��O�{��^���?�an�������N�.]������g�im��G�S��T��v 
�����G�;��d���y�������w�����33s����������=G}t�r~�s�K���s��Wg���ihh�����u��|����7��N��g���v`�=��w�����L�4)���?2��466f��%���G�����:(���s�q�e��vjs����s�	'����m������=;��/O�~�r�!���SO��G�G	��U�������~87�tS�L���3gf��EY�bE������/��������444���:���E�XKSSS������l_*.�(:�pv������EG�N�]���g^[�9l=�d��vyzccc>��O�o�[�g����v@��}��dFb��P�d�����+r�<	����IDATg��'�P4`�iw����7y���SQ��#yN8��|�c�����t���"��:uj����*8	@y���d���k^Ps2���]5z��	���]vY�=���P.:�����RQQ�A�)�������%K�$�rH�I('�����_�$I��.;�vW�>�������i���2��
���O��$y��G3c������<xp\t������������:0� �3�2	��.Z���wy���}�n�d��[[�fy�Vke��em��u�������m�]m��{-�$K%�B��	������Yw2\3s������:�|�u^s^�{����������eE����u�N8!����k���s��|&:�������/:@��X�����H:@u�n��$_���3m��������������?���k��@��������c�����46�eE����+:��6mZe{��!�7o^>��e��!�0aB��s��J�m��T*����'�����;7Ir��G����4�$-s3c�^P���^7@��6��J����tuu�����}�������n��������r�����__tjP����EG�
(:���ginnN{{{������Vtx�J�����E~=�O����2�:����s:�~���}��������_�:/��B�/_����g���9���r�	'���O.:&�Xggg�������n���������Pt����T���w�qG���/e��U��}������x�����s��������K��5e��d�����Pt����T�k/�~�����p,X�r���?������}(��[tt��^y���������fJ�R��r
�����9���3v�����Y�re�}����=;?��O����r��o~���{��s��g}�!�n��h��\z���������/��ijj�������/�����r���.�,�x�;��~�����e����������W'I���w�������<IF��o~��9����$�W�����
@����?����$��
�%�\�C������a��%I~�����h�a������/:@����HG�^P$���^7@��gR*������2�^��
��'��r����~���M�81'N,:@�k]���V��H:@u�n���?�!I��>����G��$Y�l�.�@����{��W��������/Y�$I2b��]�	�����=���E��y�#�����I'���
����������Y�b�=w�������S*������PB����g���E��y�FL��Iz@�t2��z���SNI��X�"_��w��]tQ�/_�$y��_������������2$Ir�u���3��\���^x!g�uV����=I2x����������uE�cc����g����H�T�-������y�������;6{��GV�\��f�����O~��^z)�r9�R)��sN��S������z��s�9y���r��W�T*e������;r�wlq}�\�l������g�����O��z�|��_�����~��y���7�o���|�3y�[�����444�$�V�K�5���d�����L�\.�����}���_���Y�tiV�X�a��e��Q9���r�	'��SNI�T*:nMknnN{{{������Vtx�J����E@����/�^=@'����}���
J�R^������}�Q��v�d��E��y�GN���z@�t2����	������,:@�����L��Yt����Tg�)�3�.\���1c�T}���x��5���=���J�$I�T��O?���^�?�/lM!�
���N=�Z!�1c�T=�|k���=zt�H2{��d�nP$���B�=��N=;�����$i[����fE���Pt�
��	s�����s��P�ZF���E/(�NP]!�p��|0��K�x��;����~:�-J�w�q�2}XWWW�H�8�1i�����T���@?���2m��\u�U;�����:��MKKK�.N@���@���r��cz��P�~7@_�jU�d��!'�/�w�p��G�$#G�,8	���q���@��Kg&3�Zt����TW�}���U[�|�V�Xwww���s�������eJ�R^��W����


EG ��U���
E��i:@u���=���J���/�����r�
7������w��d��^���r��gK����d����o{Q�@/4g����3��5o����>]/(�NP]�g��3f�3��}���J�6l�.�[*�2d��466f���y�����O>���_:@���6�o���I�i�������vw$jT������R��;z��<���I������Z���.��S��@O�8qb�H�����^^t����Tg�@M���/:I:�t$�@�t2��z�}��U���r�=��7��M:::�|�����m>�T*�����
)���������1i�J��F'����o���\p�Y�lY�l����;wn���#�,8	@mkijIZ�f���(:@u�r����(��sN�M��Ri���v?l�E�c/��B.����������s��\������������>��s���;������z��)��6lX��_�5�>�l�y��������
������\�2�R)�7��~��5j�f��
��'�}�{_~���3��LV�X�������kH@_�����{o�d�=��?���zN�T��>���s�9)����������=�����
��z���J�u�Q4h���]�v���s�9�k���z�����?����3�n�����$y�+^���uuu�����->w���y��^�r������=�����#���GYt��7���1C/(�NP]���>���d�%�7hoo������$������@:��^7@9rd�d��������O<�D��?���I�+V�@:������UN(����3~|g�1j�NP]��t�A)��Y�`�&������l�~��[|����3{��$�;��$Y�`�f�-�~SFM��)z@�t2��z�����J��������k+�q�y�+^�r���o�9?���6y������O|"+W�L�T�1��[s������'��d�%�7�M�$�R)����$�r9��sN&O�������?�c�=6��sOe�����������
��>���=:�r9�_�&�}�C��'��r��$�;wnn������w�-[V�������G����w����J�<��CY�vm�t�?p��|������^��~��Y�b�&�755�S��TN?����~��
�����.uu[�6x��|�3������Gy$���0`@����;,�Ri7��/���/:I:�;���H:@u��������������455�����8���.���#.���	~�G��A���G�d}_�;���.J�0 ���g2h��bPz��[��VJ�R�>�h�sv����$ISSS�Ij[�����=mmz@Qt2��������I�������d���Y�xq�1j���2i�^P$���^7@�w�}�$���'�����1��r���{��(��^7@oiiI�<����/~QpjE��}��������\.���������#P���%_���RWW����:o|�����y����C=4���?}@CCC�H2o��d^C�1j�NP]�\.������?���h���]�6�R)I2h�����^�����~J�R��������������������^�����#@�tQ��Q����|�k�e�t2�:Y����@��g*�$��r���^z)K�,��>���&��m���C/������c���,����P�&������2�"�d���3�|���#�uvv&I��Wl�7a�	�������E'���
���[tjP���;�n�tH�?z������@���f'�u3�"�d�������K�S*�r�����}��555�$m���6��H:@u�n���3��T*��s��re�T*�\.��>�m��3�����IR�o<L�
�����s�����nIK�^P$���^w��>�]���[�e����'��-���;��3C��%�\�N8��S��tuu�$���F��H:@u�n�>v��Z�a����N��Y����~4\pA���o��SO����G���;c�����W�����|����E���@�o�Ir�i�e��1y�������8�!�j��$�&MJ�\���3��@��>��>|x�d���'�77n\�H2s��d�W��P�t2��������N��]���$�&


EG ��U���
E��i:@u���O>�d|���J����~E���7����;�y�{����$9��S
N@o2g����3��5o����>]/(�NP]����y����k��Y����<��cy���*�:4g�qFO����u�����J�~^�\N�������+��W�bWG��u����;b���y�[��s�9'c����T�g�n�~�e�m��A�e������s��g���=�����
�[ZZ��@
�ut�	'N,:IZ�[�����P�t2����	���EG I����C7(�NP�������������c�����46�eE�����g�������<���Y�dI:;;�b��><


�w�}s�QG��c�������@0w��$��GYp����������z@Qt2��z���?�y��������N�\����O��Os�9����}�nH@�+.��z���y��y�������+��������f6��v���y��y����O|����.�0��
?}��9���3g������J��c�~���=��3��
�������/����v����r��n�!,�~��:��C�+|����>�<�Hep>l�����d���9����8_�jU����[n�%7�pCV�X�r��_���������~uw}\��p��������R��r��?��?��>������1�S�L��C����������<���y���d���?��r�������(t�~�UWU���������NCC�����1����2u���}����vUD��#�<2Gyd�1j��ggd���H:@u�
�.\����g)�Jy�+^��.��e����.�~���r�����'ioo�EI��
������[�.I��|������{��|�I��k���{�y��?:;;���Yt��7~����Yt����TW���G�l�v�i�d��z����X�`A,XPt��7e��L���I'�������$���c���{��}���>9��6�?l����?�|J�R��O�d�����N�\�s�=�K�@�V�}���I�#F���n���/��K�@�V�}��{W���s�$�;���QW��^�:�R)��������K�t��m���EG IGwG���I'���:�N'N,:IZ�&�����d�v	w�M
?����e���q����G�e���hooO�455���5�hN������E'�����9s2g���c��-^�8�:@�&���LZ������+|�^.����
��;���^6S���n(��`3��A����CCCC�H2o��d^C�1j�NP�:5a��qEG ��f%�t3�"�d���;���`��,X���5o����<Y/(�NP�:5���3���E��y���	:��P�t2��� �����@�����;�=�������s��7�'?�I�|��,_�<


��?����-o����Z���&M����={�N�\�hQ������������s�=�r�����z��y���?��?K]�*���e����@�t2���������Tt�>����}�?��,X�`���,Y�%K������-����}�k�c�=^��p�;��+��"����S.�7{�����s�=����+?�����}���fm���6��E���3@���^��>��,^�8I2y��L�:5{��w�,Y�[o�5�f���>��~������7����m��m������I��S��T�%K��\.g���y��_�c�9&��������w����{,�=�X�<��\s�5�d�@�g�@M�;wn�d���'�����oV�����3}��M?��3c��|��������Y�fe�����9�����:?�p��T*��o~�Ne�k��r�Yg���N��a�6y��C�����/�03g���O?������?�S�����-I�����eE���Pt�������Ut�>i��������$�x�+��}h��>��U.�~��W��������o~��$Iss�N_Z������������80�7�A�%I�����z`�4jLc�^P$��:t`��y��,_�<Ir���f���[\7p��{��I��<.�������*�;{��������>8I������k�w�[���Y�9r�V�n��/���~�r�\9�}���y���c!w�K/��$U!��c�l�{�Q��p&z5?����n�_���-���I�?��?��5{�/�P�w����k�w�[�������.���*���
�����������`����?k��M�L�2��_�����0n���7��}���Cs��G'I�|���q�[\w�w������^�r�v����+?��O�$�G���VO����k����z��vZ�����Kgf�L��H:@uuE�������}�G>�����?���ks�E��g����S���{g��%���[��o}+�
Jwww�d���������gY�bE�������z����.]��������kS*�r�E�����^���U���
E��i:@u�@�����g?��80k���7���L�:5�w\�N��o|�8p`>��OT����c�����N����o�b���{��Y�xq����x���@����a����;wn���S�����W��;���s�
7�������\��y�������!C���xEN9�������A���|��'���0w�uW���'�W�NSSS���s�i����N���}��9s�$Gyd�I�������8qb����<���Y�jU�d���9��r��gg�����{���6��d�����?O���U����#�W�^���??�?�x����}o������k[7}��d������E'��}'�q����?^�*/��b>��O���o���U�V���3�?�x�������9����{���O������7���~����w�����w�#��������c��9��C���Y�fM�,Y�5k�d�}���!C�$��_�UY;a��m����o���k�$oy�[z$��5kr�d���I�w��9��sz�����w����]><����?���w�}��9��v�aY�zu~���g�������W����Y�f�K�V�������+�G��c�9&C����>��s�&In���ttt�{��^��|iP]]]]���6���I��W�z���0p4hPN=��]��Y�n]���/��{�I�L�2%���gw���?���L���O�<����5���G����g����vj_�_~yex^__��.�,o��+����K�����+��"Ir����������_u���s�&���~�����?���W����s������+w�uW����y�����k���?�i����u�Glu���s���O&IN8��444��L�\rI����;Ir�I'�����2`��]�:��;�O��O����<c�����_���;��%K���+�������6�'����s��f�������S.����|e���o��_�je��o{.����������e����|&I��o|#�{��2r���:j�M7�����$�;���8p��o�������Swy��.�,7�xc���c��������
[�������n6<9�����\�2I2~����=�����/��=����_���[\��#���GI�0 ���������}o^��W&I�/_�n�ag�~�������/~��\z��I�8`��c������'I��k��x�������.��I�2i�������3f��k�M�q����K7��
l���z�;��������R�Tu��1cr�	'T>����o�a���}�t�I[��J�L�6-_���*���G>����[M�8��}�������:*'�xb��������=?��Os�wd��u�k�����~5C����x��,]�4Ir�����3�������f����v��gg���[}����t�MZ�[�����P�t2����������?\����4��u�{]e�~�}��S���fk�������g������9�+��������Y�&w�uW����->>~��|�K_�����ls__��-oy�.��$?��O*��=�\>��m�97�|s��o�]����5I�nP$��:�^`��yY�n]��g�o�l�?v���W��|��-�y��������_w����?~=��m>�/���Jb��r\x��y������&K�.���+�����:(�'O������:�{��������$�|�+��W�����Hc]c���?�(�NP�z/0o�����{��]���/������K�f��Q���,Y�?����c����>��Q�FU.���SO�����s�$Gyd�I��SO=5��z�������s�}����/���\t�EU���+wz�@�kijIZ�f���(:@u��@GGGe{���������>��c������������zgg�v=���[�.�-����z�^`�����������xI�b���������>��s�=���>z�k����k��6�����l1����$���Y�x�fk2n��$����K�G�NSSS��g�n����������$��9s�x'NL}}}���*g�:&���z�1-]�4���������N��19&����)Yi����-�����$����3e������H���$I���L1i�5�V���f%I&���	{L�l��e����-I�2�%��7[3s���_5?I2}��-�mmoM�����
=�W
{U����>}�{�~?�3S�l�=���>����]67�g����G����Y����<yA&L����f�N[���--s�������3�e�����4}��#Z['���>��]ii���=���v�1mxK������G8�]{L��x�uL�����N����=�19�8�������{DQW�P�������6�l��!C��#IV�^�������������*���r�!��K/�4�]vY������
7����\q������$y�k_�[o�u��Y�jU:�����o�=GqD��#�<����-����������N��_���I����9��3���
�������}��7��v�V���;��m��7����(]\*:�;��NO~�Q��	=�O����2��t2�9:Y�����{�Q������x���H�a��m�~{�������`@�9��@c��~N��%K��9�?�|�}l���%K*�?�5�=�\e{{���<�f<;#����c�4��:���L�0���d���:}���������5j����{����^���>��6������K�Vno|�x�����0aBX���\.�7���6�����V�>��-��x��_�z��9p���?~���+:;;���Yt��7~����Yt����Tg�����������~�����|��}�	'lq��^������I�2d��m>��X�`A,XPt��7e��L���I'����x���T�nmm������{���s����{���E�����_��SO��Z������6mZ��c�$��y�r���V]{�%�d���I����~��[\����&�y�k�$k���W��������k2��$����3m���9�>����{�����|�r����|n���M�tww��K.��7�X��3���V�{�T�����K.Iww�&kn���|��_��>��32r���9�>���}�{������or���?_��3gN�L�����������i���=������}������|�������v�aY�zuz��,^�������d�?���f<���r�9�������$����������1��!C���G�O<QY��'��������~��e��o�g�}���+W��c�=���|��
�����|����-���$y�������o�������g��]9?��Oe����������������nO�������������������|�Ikk�I��)��N@_����t����d���e?��Y�{�V��<���������������_��~8�/��A���~���SN�����|��;��O>�d�����}��Y�hQ���3z��u�Q�6mZN>��������������������}��R���?i��g���y@��������L/���A����>tvt��3@���M���A-�?�������e�t2�:Y�7���;��������5�yD�^P0��:tj����3i���c��I#&�e���3@��@tHb�I���������v�m�5kV�|��,[�,uuu�g�}r��G��xG�<��->��[n��_�]���/~1o}�[w:����s��������y��t���J��5*�~������%'�x�N���2o^�)��r^2���5M'�������'��sN���������y������O��[n�����O~��)�J�������n�m��-Z�(�-���3s�����K.��{���������YE�`���Y���P�t2����~g��5��>������3n���X�"s���5�\�U�V�����>�������U��/��/�{���>>z��������'I��k���
oHsss��o�80s���~��,X� <�@>��O��+���>}�'����~������#��7���8���q���O>9���5k������{�����m�[������G��=:���g3u��<x��^��W��o~s>�����G�#�<��������S{$K�`��L��������<rr2y�^P ��:�1���~��������M��z��9����$/��b~�����x�����w����
�7����\P�}��w��h�Nggg&L�,:@������`:@u�@����]�3fL�uc����sz��:(


I���3@��q����C[8?���I�R�������s��>����t��y����+��"+V���W_�N8a���?�����{+��^u_|q,X����6,����9���v�i�w�}{�X6d]�bE����|e��@�r*#��444�o��oS__�9s������r�����G�C=�+��2��OOwww9���{��[�_[[[�,Y�5k���C~��_����v����<���������|��=y��=�Z�t�_:��Sr�5���k��M7���.�h��G��3�8#������-�c��1y����?<�G�N����?��Or��wf�����W��R��w���=rw�yg����$����7��
=�:�`����=���^6;�=��5M'��T.��E��hnnN{{{������Vtj\www�����x��������W��U��?�SN9e���/_�a���T*m����sO>��Oe��5�����7�����{���~��������+2d��\s�5.��2U��	�L�$�Y�bo�#.��=A/�?����d�#t���%��~g��U9��3���|'��-�_��_��n�<����g������5��c�=�O~�����k6�����������N��>��$IWWWn���]z�?�|�9���X�"�R)_���z�:��\y����/�$���?���>;x`
���������7���L�4)�r9_�������v�_���|ge������������},�-J�|�S��������_��������E��y-�[�2���d���J�\��7��$9��2u��-�����g��$Y�n]n���~��#Gf���J����]a��9���2��$�g�����]��Z��������c���A�z@�t2����~e�������$�8q�V�z�������m�2�;���+���'��c�%I����V.@�3@���������k��v��5�����kutt���3I���{����8��?��������w��s�yY�`�����#2l��$���>�����m����������~�����$�QG����`������>����?I��7�9\p�N���c��+��'��d���������e�������+�O:�����E���Ol�u����|�[�J�2$o{�����#�H&M��I�&e��E�=^.���/})w�yg��
oxC���/��K��}����o���?����+]]]���+����g���3fL^z��<����������'I�9��w�q��/Z�(g�qF�8���t�I9���3r��$���s��w��;���}~���f�}�����_~yn���$��	��| ��������:h�^���7.3g���Kg&3�Zt����Tg��;x`.���|�s�Kggg�����s�=[\{��G��_�rQ���~�_��WU_���>��w^���w�t����'��y���=�y�6�3{���~�Z�������N��U���
E��i:@u�@�t�����n�M7�����?�����/���f�����^���z��9��S6�\����������~��<���Y�dI:;;�v���1"�����G�w���3���J�
� �����9���ijjJ[[[�q61g��\qE2c��EG�~G��?+]\��"`�L;=��G�2�z��^��N=G'��zt�:$1@�$��$�+:�'NLkk�)hmoMZ//:@M���3@�&�������t��H:���P�t2��\�����������c����F��`:@u����s���en�1j^KS�^P0��:t�:$1@�$����$uE��tq�����(����xvF2�E��i:@u�@��P#����;��P��2���d��P����)S���e����3@��@tHb�I���������t����e����3@�&�.nMZ�+:@������T��tjD���47����e����3@�&L1)�&-.:@������Tg�1@�$����$uE��a��y����c�<��x:@u���Y/�Jf�+:@������T��tj����3y���c�<��x:@u���	{L��	�E��yz@�t2��� �����@�����;�^6;�=��5O/(�NP�:5�mY[��Tt����O'��%� �����-ii�[t����O'��%��	������c�<��x:@u�@��@tHb�I��3��������P��2���d��v����'���P��2���d�9b�@��>vz�O�St����O'��b�I� �:$1@�$I]�`whmoMZ//:@������Tg�@M�X��t����e����s	wjBc]c���P��2���d��PZ�Z��2��5O/(�NP�:���@tHb�I�R�\.�����9���ijjJ[[[�q������#@�s��<�	�$��N=D/������e�t2�:Y��t�:5b���?���5O/(�NP�:5a��)�2eA�1j�^P<��:t�:$1@�$��$�+:��IG}�1j�^P<��:tjB�������c�<��x:@u.�1@�F4�hNss{�1j�^P<��:tj���2i���c�<��x:@u�tHb�I� IRWt������k(:@������Tg�@M����d���c�<��x:@u.�1@�FL99�'/(:@������Tg�@M����L��Yt����O'��b�I� �:$I�����e������P��2���d��P���%mME��yz@�t2��\�b�@�h�����E��yz@�t2��\����8�1i�*:@������T�t�:$1@�$����1s����9��5O/(�NP]]�`w��j~2���5O/(�NP�3� ����c�g��9E��yz@�t2��� �����@tH��v��������c�<��x:@u����5IG}�1j�^P<��:�p�&4�5�����5O/(�NP�:5���%--s��P��2���d��@� �:$1@�$��$)���r�!����������������+]\*:�?y����M���d�C�2�z��^=@'����}�@��P#����;��P��2���d��P����)S���e����3@��@tHb�I���������t����e����3@�&�.nMZ�+:@������T��tjD���47����e����3@�&L1)�&-.:@������T�3�{����.��w�=���~w����������sO����<���ioo��!C��W�"��rJ���w��������z
x����O:7�|�&��Z�*���y���s�UW�����Yg�UPJ�b��RtPN<��m��4i�V����?���w�}��9��v�aY�zu~���g�������W����Y�&���'^v~�����z�k_�/��/{?�_~yex^__��.�,o��+����K�����+��"Ir������������_�/Ptz��%Kr��WVn_t�E���d����������-IR.����|e�������y���P��2���d���c�_}V�\�$?~|����T]{��f���_mmm����[2�.�^��Y�����e����3@�������vKKKJ�R��c���	'�P�}����h6����������?\��=�i��������s����#'g��E��yz@�t2������-[�,��rK~���f��e�s�=3z��477��C����I2o���[�.IR*�r�a�m�5?�����O>��������dB�KSL/(�NP�z/���8?������+_��|�c�_��_T���7�����{���~��9f���vggg�.]�Q�F�`r�������~�O~�����;r�Wd�=��lMGGGe{���������>��cg������E���f�������$�^f��1�:ujN<��r�!5jTe }�������v�z��$��Y����},W]uU����W�\Y�������n�}����{.G}�V�����f�y��I����`����R__��'&I����x����444d�����Y�`A:;;7[3z��455%I����������7.


I�9s�l�&N�����tuue���[\�r�i���I�y+�e�����\�	{L�l?���N���$I���4j�l���33��M���Z�[���#�u�iij�����H��:>SFM�������.nM�4�h���6[��S���a����/VnO�� &tn~L�G��m������il��{����2~��c��������Q������l�{����G���)S6���Q�����#���3i������k�\j�19�������n��z�c�u���{�crL���^5�U���n�^�=�19��L����{n��i����1�s�cz�����57�{�uL����/��n��������y����i��m6O�	&d��	y����.� �]w]�����������y����������]�?d������J�r�\tv��u�r�i����J�r�!���;7Ys�W�K_�R����}mn���m�w��U9���*�o���q������9����w�}s�m�mu�K�o[��-�=�s�G4'?�������M���d���e�s�2�3�v-�z�N��9�0`@>��O�/��/�$O<�D-Z��Pz��E��3��x��>[}{���M�����N/(�NP���
�O8���2h����
���Ac��~���%K�k��?�|�}�w�}��A�2r����^xa��'L�P�^�d�v���p���vCCCF����-�[��2��5O/(�NP�z�r����_n}��	0`��r�����7�����>Z�>���wQJ���qPc��#-�9z@�t2�������_��=z�&����������~�����|��}�	'���}�z�����=b�������l������vkk�V��p���{��[|.@-0@�%V�X��k��_��+���~���������M�6�ri�y����k����K.�$k��M�477�����<��z/q����-oyK����,[�l�k���r�UW�/��/�����I��k��w�y[\���{�#�H���?���|��������%�\�o��r�g>���y4}���-S�Gy$��{n���r�Ae��	ihh���k�������M>����>����7�����{����/~����/]]]��G?����9����z��<��CY�xqe�'?����=z�E��tf2��E��yz@�t2���{�5k���'��O<Qu�k_����?�c>����k��A�����O�����[�$�?�x�����w�y9���_��B�W�O�7���e����3@�%���wd���ikkK[[[,X�^x!)���s�=�����������Ss�1�l��G��o|������\��y����x��4(���_N9��������0�?3@�%����>:G}t����'���O>�����M;=�>'3fYt����O'�n@��70@��@tHb�I����������^^t����O'�������#��/:@������T�������46v���e����3@�&�4���en�1j�^P<��:t�:$1@�$����$�r�\.:�Csss�������������y��KEG���"oy��I�3�z�^=B/�?����d�#t����tj����3~|g�1j�^P<��:tj��QS2e���c�<��x:@u�tHb�I� IRWt�:�;����c�<��x:@u������I�uE��yz@�t2��\�b�@�h������c�<��x:@u���I#&e���E��yz@�t2��� �����@�����;�[9/��Pt����O'����0��Y��qE��yz@�t2��\�b�@��<rr&O^Pt����O'����0a�	�0���5O/(�NP�:���@tH��v���f'�G���e����3@�&�-kK����P��2���d���;���2�%--s��P��2���d���;5�qPc��Ut����O'���tHb�I� �:5b����9s\�1j�^P<�������0��d~C�1j�^P<��:g�@����N���s��P��2���d��@� �:$1@�$��$�+:����I��E��yz@�t2����	k:����c�<��x:@u.�@Mh�kLccW�1j�^P<��:tjBKSKZZ����e����3@��@tHb�I� IR*����C�?477���=MMMikk+:N�W��Tt�.��=A��?�����e�#�2�3�z�N=B'����1@�F�:>��w���e����3@�&L5%S�,(:@������Tg�1@�$����$uE�����#��/:@������Tg�@Mh]���^Wt����O'��%� �����inn/:@������Tg�@M�4bR&MZ\t����O'��b�I� �:$I��������y
E��yz@�t2����	�^���Wt����O'��%� ����#'g��E��yz@�t2����	���	:��P��2���d��@� �:$1@�$I]�`w��lv2{t�1j�^P<��:tjB�������5O/(�NP�K�@��-�[��2��5O/(�NP�K�P5&�]E��yz@�t2����1@�$�����P#f.���3����e�����+:��W�O�7���e����s:��1}��L�>��5O/(�NP�:���@tHb�I����������^^t����O'�������#��/:@������T�������46v���e����3@�&�4���en�1j�^P<��:t�:$1@�$����$�r�\.:�Csss�������������y��KEG���"oy��I�3�z�^=B/�?����d�#t����tj����3~|g�1j�^P<��:tj��QS2e���c�<��x:@u�tHb�I� IRWt�:�;����c�<��x:@u������I�uE��yz@�t2��\�b�@�h������c�<��x:@u���I#&e���E��yz@�t2��� �����@�����;�[9/��Pt����O'����0��Y��qE��yz@�t2��\�b�@��<rr&O^Pt����O'����0a�	�0���5O/(�NP�:���@tH��v���f'�G���e����3@�&�-kK����P��2���d���;���2�%--s��P��2���d���;5�qPc��Ut����O'���tHb�I� �:5b����9s\�1j�^P<�������0��d~C�1j�^P<��:g�@����N���s��P��2���d��@� IRWtv��^z)7�|sn�����;7K�,�^{�����?o~�������#G��5������g�����7����s�����K[[[����\v�e��?���R�T.��E��g-Z�(o}�[����$)�J9���2n��,]�4��sO����$�
�5�\�O<q�_���9���ijjJ[[�.=�ZT��Tt�7,��k��$�����u���m��L'�]K/�������e���d��t����5���xex>v��|�����_����/��B>�����{�Mwww�O�����?{��WQ�v�E�g�y��y����$��w���M��I2r��|�;���q��$����������H���w������i�r���nq�{��O~�����\sM��Y���z
�~l������*�[ZZ����o~s�
�d�Y�>�`���M�����gg���I��a����f��������\������3@���|����!�����m>����l?��S=��72@�����W�;v�v=g����l��d��$�guttT���g��z����[�������]�n]-Z��5�z�~l��������z���6~��X�dI�������G����F����A�%I��r��Y����R�r�u��e�����0`@�$Y�vm��[����f���_X�fM���fk���R*��$���[<�
y�g�������3��d����e?d����6�g�����]^�a���P���]�����~���b�UkWe]�e@d���[\�b����������������Z�*I2�4(��l�crL����Iih)k���>��%���6uu��x�����^C��������������G����V
��u�����[^�b����S_����u�JY�j����A�2x�����������sL���cz�k��{���G8�]wL�_�=�19�]yL�:t�1x��IJ�{L�s�cz��t��9��{���G8�]{L~V�x�uL��e���YY�����Iw�����:&��k����o��n��1h��������������R*o�H�ZZZr�}�%I�=��|�S���s�����~��I���x���������-���QMMMikk�����~l��!��j����^z�����������^�:���#��v�o�l��?����c�����a�����9�������������������������?�]�y���*�


�:@�e���M�0������n�s-ZT�>���vy&����;���+�O<�D��Y���<����mt����c�&M��!C�$+W���9s��~���y���+�O8�������c��
�d��������~{�/_�d���w�q=��71@������U�������;w��V�Z�����~�{�������[��s�'O����d�%���������d�/��|�����d���g�y���
P(��/��/=zt���g������v�i��'?������1�����;IRWW�o|��k����P��S����{���}^w�u[\s�u�U��{��������c����3�<����;��;���K/�|����K������e�9���p�	9���|��e{��g*_C~Q}W8���*�������fG��_���o�����7���=�[�#t��K��?s����~����5g�yf~����\.���<���Q�F���.�I'�TPR�_��v�&_����_����2d�����Ks�e�%I������+����;bW��i�����/����O�����w�yU<xpF��<0���iii�!��@q6�Z��w^�?���~��]�����
7��#_����:^xa��[���x��gr�q���T;�w]k6����������^�0aB�9���~��9��vcB���:����G?����g?��}�c&�2k���v�z�8���r������n�M7���s�f��%1bD8�����o������#G�h��E���k��~��(�/��R�,Y�%K�d����1cF���w��K.��������g�}6_��*��q�����}m�
���������,^�8���������p.���<��xP�������o�������W��}t,�������g��i�6mZ�Q`���?�s��/�2C�-:�Ny�������z���{��I�������N�����:jWE�������������ikkKwww��?��?�h��\{�����o�1k��I��r�)��������_����o��=��#y��G�$MMMy�������y�kv8G�������+���O��GI�\��u�2c��<��s�����J���@�n����Z�j��6�g����cK��>�@�����������?������/��[}��������������g�}6g�}vz��$�<����*��Gwg�����O����^t��>�he���N���y�466n��]z����������v���}>������>��s�&I��?�3��zj���������������Tu���W�������U���
�u�b�X����j`[6>���+���/�X`���c��{���&����o~��D����?����������r����?�AF�Q�O�V=�����J�R>���W�����K/�>F��_�@���w�+&LH�tvvf��'��>|x>��Tn/^�8���oL��
�oO��/o��W�b��'{��G��-5��nH�\N�w�qy�{��Q�F%Y�s�Y�f�>F�X�%��^o��9���s��g&Y�����2r��]�mmm���~����?�/NWWWs�!�d���9�����{�����>�������n���c4k��466V]���O��o|c��~����g�����N;-<�@����z��^����\{���������Og��U�w�}����.���s�!�l3kwwwn����~����o~�%K�d�������>���q�����}m&O������;�7�s&M�����O��O�$����?���p�
����1c*�.���<�H����<���ioo��!Cr���MozS����g�=���}Us�u�����K�L�6-�_~�V�\���={v�y��,_�<�
��#2f��v�ay��^���'o���E���k����g�������~����O>9g�qF����������n�)3g���~��,]�4�����{����:*o}�[��7�i�?�k��e����s�������}���������;.�}�{s�Glw6�T��w���z(O<�D-Z�U�Ve���ijj��G�i������e��������K7���l�6E������y��I�W��U����2x���������w����3f����S7Ys�����g�M�<������w�{��+����9sf�x�����Y�fM��}��7�|�+����7�����w�I�&����J��]�6.�d����$�<�L�;��$��P��L��q�=��������#�d����c�=r�A�mo{[��/�2C��������^��.�,Ir�y�m�R���{o��?�3�<�H-Z�+Vd��Aihh���?<'�tRN9���~�m����������u�]Y�hQ����?�g���������\�2�_}~���������K3p�����>9��c��w�3'�x�v�o�����w��Y�f��g�I����{����w�q1���?I�D����z�l(��vW+K����X�,�xrh)l��c<J���jk�6�6aISRJ��T9t>L��c^�;SS3����~��^�=s�u_�����u]���055U�-"���D"��GQV�f�����ki����_�W�c���K1�X�����t"z-|��W��a�������M�6a����n^^����c��Ux.==�������������������9k����[�n!==s���4�RII	������O>�S�LQz���hL�:���r�?|�>������WW�J�HNN������S����\���"%%������k��_�_���������~[n?''���]&  k��Eii��X~~>����={�`�������������o_���b1


�����W�"((3g���y���{��)���x��������HNNFHH�n��(m���K���BJJJ��=z�G���������m��:T��H$��i�*|�����{��a����={6~���}#""`�����u��"J�������s��yAAA:t(���S=�{���0`233���???,Z�H��999�1c�pcw��Qn�*R��111�2eJ��F���Fvv6����U�V���5�9_����T�������S!^*((�H$�H$��?���;w
��)//���8}�t���b�p}#��s�N�^�c��Q����{���(,,�{����q�����}�����Ri[�����/233+<Wv=p��0����K!����'1g���
p��m��}AAAX�l�����oDD�#�H���><x0i6����g����������1V�`�%�1�?c����D�Z������&L���{7&O��������#G�����c������
tuu����H�X���L�07nT)��K�����a����B���"88Xa����X@�V���O?)]�{��-���!77m�����
Z�j���t\�x���?>�v��pYNN�������f������)tuu������t$%%!++��W��|@X+��c�����Wccc���ZZZ�y�&���H'u�;�����Z��t�R��������
�[��D"��g�p��m$''����0�|��b��?D��-���CDFF����:u*�������w��q��1������z�B�����I��wW�\AII	bcc��W_!44���S���k�0v�X���
�YZZ�k��(..Fll,RRR���_a�)����D


�������Z�B��M���3\�~]�u��Qdggc���*g
���o_������,���_�}��r����m��A`` ���D��������������n�c���J�������T���
�
���`ii	ccc��� //��7����-����������m��/��n������			B9�;w�`���8v��\�"u�1c���]cccXXX@__%%%x��)n��)� R�����?LLL`ii	mmm��{���B\8a�DDDT�l��
K�,&��l�~�!:t��X���o#>>�aaa1b�=Z�����0L�:U�h��	�����{�!77QQQ�������J�7DD�#�<�������={���w��Eqq1~��7L�4���Ygc��X�c1��7�������+++\�zX�n�/_^���,Y"�kjj���'N���y��=������k())���',--��s�W�yjS������-��]�hlmm�fa�D"�_�^�_�z�J7��.]���������Mn�kjj*����7o���+W�T�>u�������������+L�#�H������DGG��W5x���-���6V�^
ggg��d3dgg���'O�����Z����%�������?\\\hddd���Wi�����[oa���pvv�k���[3f��������k�

��u�<<<P\\


L�23g��0����xxx@$!--
�g�FPPP�����0s�La��c����ys�����y�xq@D��>�������+M=���g#%%���8|�0�^�s
>\x]YVpss��t���������c�H$xxx ,,L�����#���_M�6�����U����m��	7vmmm�i�&��cII	���\���"&&F����D�������};Z�j��7��O?�{�����9s&����,Q�2���������S]]]l�����Y<��#G��m[������h��
~��'888�=w��e|��w���FFFv��Yi��.`����H$h��<==1q��
1dbb"f�����o����X�d	���*�����9s�7v�u����7�K�.�1�����eV�X�������|�8qB�/C
>�V� ��}��X��1�?c1��7W��5s�������D�����_~�E�_�d	&M�$7x���BBB����l�������&\yyyr�s_�x����C,\]]1h� ��-,,��+0e��
��

�q�F!����DFFF�6d�-^���9���'����>{TVnn.v��-����JQQ��YSa����,�v�q��p��e+�`���9rd�k�8q"F������"l��
��������pa'N��P���QPP�������
722��}���2<<\�� ����Bimmm���(L����)�Y��^'������Q�_xx�J�N�6
�F���n���-���mmm��>~x{{�[�n�7===��=z$�J�9s��g��*�[��"�H��t�e��M��O�_����*��6<~�Xn����e�f*..F�&M�{��
7v���m��	���������������SFFF�5kUj{���n�������/�9rD��KKK���?
e�6m�www�0-,,p��!kPHH�0W��m�������];���_��. ]-���^^^����Q:u���B�M�6������������q�F]w��0��9�X�c1��7����boo�>}���5��sss�?��c������-�9r�B���@CC����,���x�^�0�|����N
X�x���v��
c������]�
e+����'��jn����7n>|(<V��mmm1t��J�733����sv���U��Gu
0@��A��>��������AmY��_���H����'Wy�������C�����*"lO�0��	��D]��'"�Kqqq��g������j���;wV�����[�����[oa��M��?��{���X,��3�������OW������}Z�v��M�������bu����amm]��������/����`��E]����+�w�^��#F�&�&''+��=s����@ZVA�D�w�yG�����q��q��%	8 �{xxT������7:u�T�9��^G�����
����S�N���Sx|c��fcU��1������^;�����a�H�www���{�j�l@��k�4���x��9
q���*����];�Y����l�����8v�i]��7��n������cnn.��W�@6�SPPV�\���k�������#�Xaa!RRR#������i�i`�����0�6>>yyyh��y��E��:y�$f���R�)e�R�����;233Hg���/#��q��a*�1�����m����t����U��tfDD�\jj*�^��{������(((j�&�I$$%%������Z���}���
6/^���8����o��u��U��T��������pC.((���*��6���B�������o#..N�����''�:��z�
�t����"�:t������+�����E!99���w�����)l�]�)#�EGGc��)���;w��n��M���_W�����
�
6�tn"������q��a��8#F�@dd$��r�:]�\�c)��zcU�15v@'�����5������p��b`���*�^"�������*�I���`ee��g��: ��:q�D��������w�����>���j�W>8S�U�V����NNN��?��ABB\\\�����_��W��S999BM��|���X�b��*�.}���J����tuu����X���$������:v����4�������F����aeeU���]�vUz��gIv%|�+W���.]R�����_��T7n��Q�h�fffJ�S��""z���=s��Q�����������������W%++K��4������gq������Ii����urr&��X�������3���['u0�����opss���w�����@�^��'7<y�000��������7G^^`oo���G���>���E�Kj��$[B'44�/_V�������[bb��mjj���Oy�����9|��p-kjj
KKK��
<>>>(((@ff&"""��g��eW�c��cUc��1�����D�Z�;w.��=�D�c��a���*
��@Av����fd�k�7v}||p��%��ySx���>��i����*u�����mE�����'����e�_��kkkk|��G������b�5k===�w��1bD����M��6


t��w����������[����#77YYY��y36o�mmm|������g�}���{�|1�������s���e222�mUk��z�������O���;���T����M��~xzz�<p^F���7��5k0`���:�*K�T�6�s������:u
��_	�%


akk+�o�Y��*���������	lll0j�(����{�����E���i����((( �=�qs�u�������Y�P\\���4   �����������U��${-������.l�e��e��*�Qc#�������������������o�:�K��*��j���1c
t"z-���C��RZZ*��VEnn����)�e�+�FC��Y3����=���X��Tejk����K�c�XYY�=������b����������z���S�NHMM��w��}������c������s*����s��O�>8s�F�!�����D"�_���
����p��Lm|�dg���X,����]��{@DD���}��������d��������Grr�����������To����=��_�����������;������_����T>|������<==����W>��>}�T�����q��U:ts�������qC�c�y1t�P�8q�
�����������������1h� DEE��fm|�e����t���...N�����gg�*��`?s�L�A���1�<�X���r���1�
t"zmyzz���S(--�����8���S��������������f��
����?�A���abbRO����4hRSSq��%���@$�������CCC��G��k__EMn�����4�V�s�������?����� � ����Xa���{����___��I�"���c�a"MM��.������������c�pC�_�~��{w�%@�����h�"��wO�1ooo��������4440z�h�=����|�2���!������0!!!�����c���M�z����n��j�P�Xu_GXXX`��x���������k���k�����7n������7&A����Z���7"z���>�H$���U��8v����[ut�Ab�U�c�c,"��
t"zm���a��a�����^���'7[P�������+�VE\\���A���<��1Ca���fhh�Z�
g���H$����0����gX�xq=���R��W�U��|%�\z&�ZH���ys���s����������[���MX�r%?~����k�N����|��d/@?~�R���5������4����NU���|��&:u����@�*�����t���g=��&&&puu���kq��%�?���P�1%%k����^�C6~+��F����y���J?������@�]G����ptt��p��1$$$`��5B�M�Xoo�j�����m�
�����1~#�����G�y�6d�;�X��1c,Y���1�:�����#�:w�\�U��hhh���\����Q�����f7 M!����a���� ���#q��a!�������}vQ!CCC���r� ��?����z��?dWz��:K����bccUj�,m�������.���`��!8x��0�]TT�s������%T��U�[�nBi���l!BU�\����%"j�dkLv����c_�|�7n��K
Vzz��
\���G(��� ((���W),Z�s��;}�t=�H^u���~%�J��l���];�o��Z��--[���Q�����o�������kd�����WnOvu���wU���������0<��������J�d36^�r������!�X��Kc,�X�xq��^k���5j����*��?�X�>x���U��N��Bmmm|���5���-\���� ��Y�l��m+�"}��M�t�R}u�J����vqq�pQX�:u�$�&JIIQ:�����>�������C�	�����^g�U�V�����<y��s0@�

}�s�h�����~u�""RL��������
"CN}�H$�������{c��Y���.\(�x�b��{���Y%�����_=�D^�������$��~]�o}�Q��Q���������{&�8p@n�XM�����w� ���l%fII	�=�J�$"j(dW�;88���*������&56��Z�X��K1�XD�����7k�,a�_TT"""��f��1��������/����,_�\�:t(���^��jp��	����t������5�w�}@Z{h��Yu:8�����q�i�4iRg)��i��%LMMH����Wzlbb"����}���H�8q�������={��c����9T�����g�������
}�� ��6s�L�����"�����G�;w��r&���G!���i"�7�����]���{��!00�.�� m��.\ �7��_/��������b����>}���y��&�[C����Ka�TFFF��r������V���r����*����M��K,������l���r)s���U*����jk6i�Dnu`` �>}Zi[�l�����q"�����8{������\�����������u�K}c)�����:��

1v�Xa_�4?���r�Y�`������R������1c��@��-��?�R�kOZZ���'�{xx�W�^r�,X�@�	��������Y�����������J/J������!���'J����a��	�~~~
Q���1z�h�Uw�j��f���pvgLL��#�$533�����}U�����?�\i
���\�\�R(m���	{{{��I��e���!������s������������V#F����	�������w��a��=�A}.��*���/V8����pqqANNN�gU����X�j���l�2����r��Y�F(�����r���`mm��s�"22�B�\&>>^nW�����{J5m�NNN����W�R-�����S�	������X,����q���
����a��I�ML{{{����?�j�.]
ggg<x/^�PxLVV����-[�D������2������.8�q�����;��&11��/���������P�433�|�M�������eV�Z������~�M����E8�Z�6l�p�"55/^��>6����1VE�������"��0c�+M*�����]C\\JJJ����
6������HIIATT�b1i����/���!(�����3g��p���6l�'''����ppp��Y�.ei�9mmmt��
FFFh��^�x��">>^8^[[[�B�!�8q"�������/^������055EAA�]�&������h����E�������������n��%R���.��]��5))	����������v����[����������Ke���CCC���L��]�q�FL�6
����z�*���`ll������O�>ERR����l������u�������<������	VVV033CQQbcc���@z���>�DD
������O�����puuE�=��Khhh 11�n����m��Q)UccQ~����C1b��
��i�k�����~@�����A���`��}��oZ�hsss�y�����Brr����u���j�Wu��5G�E^^������;;;�������ALLRSS��iS�\�R�N�2���/�c��=��w���9$	��o�z�ju���D���(DEEASS���055���>������������.����Z�U���~~~���!���g�����G�n���E���#33IIIU�v���[#  �&M�X,FRR`cc���{������������.~T""��M�>h��j�
744���-._�,���'��j�c����1c,zp���v�����
6lP�5:::����9s��x?~��v���������2�7oj����a��u�������1o�<,Y��t����\�Uu(K�H/�^���W�*<��w���u���{w��������g��3YYY�H$�Dr+��5k�E�a����@�4i�={��~�	���j��j��=�l��>���������
)))��qy��5���3�<3���q��Qxzz���kPe?i��:(|�g��

��i��U��?�M�4���&N��t""%��m�]�va��	B����$$$���_ 00�����x�bae���!V�\Y���������k�.a�dXX�����G]]]a�\NN���*=�{����y3��o��>UW����m�6L�<���(..���i��%��Y�=zT�� 77!!!HJJBRRR�cLLL�k�.t����?�*d�m�X�[�n��`.���/\]]����\]]all�y�������H$U��f^�������M�6���/_�Dii)._�,���K�,���~����!�����HLL����]�ue'CCC�b�
�{�;�Xu�1c,�X�&�:5��M���{���K�_����-[�`��I���_����u��033����7�4���			r3-�/_�t��������?999�>}:~��74m������O#66/^D\\���������|����]�v077���#����������������m������:t���=��/���	///������CTT222���ccc4�}����j�'Rl���<x0��?�����y�=BNN�4i===������?�����/V*cnn��'O���s8u���������/_�Y�fh��
LLL`ee�O?��{��2����"""�g��<y<@qq1`gg��c�����B"��[���q��Yl��aaax�����=z����������w��i�N�Z�~���w\�t	7o�����������������������x��O�<Aaa!ttt��C�������#�4i��p��9l��HKK���&

1p�@�7���x��Q��������?�������#33:::������+����Il�l�2�?.\@ll,n�����4����i�����������1b��m�V�}R���?b�?�����x��	��������m�����{����,,,�lo��!���7v���3g��������:t���}�b��q���K��_"��Dv����A�W�<.Daa!���p���:�LXc�-�X��cQc�!)+ADDDDDDDDDDDDDD�k������������������������������������t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�t"""""""""""""""@'""""""""""""""�?���yO��IEND�B`�
pg_recv_perf_v6.zipapplication/zip; name=pg_recv_perf_v6.zipDownload
#54Ajin Cherian
itsajin@gmail.com
In reply to: Ajin Cherian (#53)
1 attachment(s)
Re: Proposal: Filter irrelevant change before reassemble transactions during logical decoding

On Wed, Jun 4, 2025 at 8:22 PM Ajin Cherian <itsajin@gmail.com> wrote:

On Tue, Jun 3, 2025 at 4:25 PM Ajin Cherian <itsajin@gmail.com> wrote:

On Tue, Jun 3, 2025 at 3:55 PM Amit Kapila <amit.kapila16@gmail.com> wrote:

You haven't shared the exact test scenario, but I am assuming the
above tests are for very large transactions, as you are comparing
streaming and non-streaming modes. Can we see results with short
transaction size (say one insert or one update, or one delete) as
well?

Attaching the scripts I used for my tests. Yes, I used transactions
with large inserts. I will redo the tests with short single inserts
and share the results here.

I redid the tests with 10k small transactions (single inserts) and the
results are not great with the patch:

I did some more tests with 20k and 30k transactions. Results attached.

20k Small Transactions

No transactions published: +13.9% improvement
Half transactions published: +0.9% improvement
All transactions published: +1.4% improvement

30k Small Transactions

No transactions published: +15.9% improvement
Half transactions published: −1.3% degradation
All transactions published: +1.3% improvement

regards,
Ajin Cherian
Fujitsu Australia

Attachments:

2k-3k small txns.JPGimage/jpeg; name="2k-3k small txns.JPG"Download
����JFIF``���ExifMM*;J�iX����>�Cherian, Ajin������55��55���2025:06:06 21:43:422025:06:06 21:43:42Cherian, Ajin�� http://ns.adobe.com/xap/1.0/<?xpacket begin='���' id='W5M0MpCehiHzreSzNTczkc9d'?>
<x:xmpmeta xmlns:x="adobe:ns:meta/"><rdf:RDF xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:dc="http://purl.org/dc/elements/1.1/"/><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:xmp="http://ns.adobe.com/xap/1.0/"><xmp:CreateDate>2025-06-06T21:43:42.546</xmp:CreateDate></rdf:Description><rdf:Description rdf:about="uuid:faf5bdd5-ba3d-11da-ad31-d33d75182f1b" xmlns:dc="http://purl.org/dc/elements/1.1/"><dc:creator><rdf:Seq xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"><rdf:li>Cherian, Ajin</rdf:li></rdf:Seq>
			</dc:creator></rdf:Description></rdf:RDF></x:xmpmeta>
                                                                                                    
                                                                                                    
                                                      <?xpacket end='w'?>��C

		
'!%"."%()+,+ /3/*2'*+*��C
	
***************************************************���
"��	
���}!1AQa"q2���#B��R��$3br�	
%&'()*456789:CDEFGHIJSTUVWXYZcdefghijstuvwxyz���������������������������������������������������������������������������	
���w!1AQaq"2�B����	#3R�br�
$4�%�&'()*56789:CDEFGHIJSTUVWXYZcdefghijstuvwxyz��������������������������������������������������������������������������?�F�(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��+��o�4�X�w�����I��d;����:
+�������i�������h����=9o�K��-��|g��#�>��4h��(�Z���hV&�W����0S,��A=j�3Gq
M��E��4�(��
++���>����u�Eh��8�����;��9�C�OX��IW�	R9q�A�q�@tQEQEQY�����?}�\E$�Y@�:G��g<f�4(��x�xf�Z��X!�V+���b��������(��(��(���S��;��#j��J��"�LQ�l���V��}���_Z�0����Ypv����@h�x�N�����������LQ�l�N_����4�k�R���h�.��4f�(��(��(����j	�������Gi�����T�=�������^�Z������G6��k����E�������-S\{�7A�~LL��s��h������	�UG����������L�,��>+�.�+�Y�RX� ��J�EPEPEbx���W�ts��R�F\Gq����e@Mb�G�L^-���W�Q	�n.�G����;Z(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
)���B���#E,��@��X>�}�����;�,�v�g�<��Y@$����h��?�'��J]>��{��X��Z��Q�bx��a���u$������C�8�a1o=�=3�@�W�����}��jW�z�1����u�{��aEr^����\���s$W�����~\����kx��0xW�7���2�
��4p�se����P����f/�v�W��H���fH���3��3Z4QEQ\������.���_\BJ�,���t������C����<���	��"����~��::(��
+��w��?�l���Fn�c�"��q���~u�C���/:����F�Y�����=2����6���
��q����9cl�
��G��o���]�#����C��94�Ey�����:���Mua$��7���F'���~5�TQ\��|sg��[��m.-�����'��8�F�x���
(��
+��~2xKB�d�73���J��0�����3F���gR��f����qIul�����MzQ@Q@Q@W7��Zx��V�����������$1��@��@%���Qi���d�G�cpc\n .�����/[���V���[�q�lscp��	�j�+������g�'�P�lX����+�?�}�/8�5������~-xg��F����p������c�zp
v�QEQEQEQEQEQEQEVv���Z�����Z���"l�m����S<K����
�k7qI,6q������q�H�R������|9e�Z�$P�G�"HF�#�;V�QEQEQEV7��B�����-���6�ic���.p[�#�W��N�Y�m�	��Q,����tQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE���W���W������{����OJ������=:/O��b����X����h��"�_�?<S��E�����*�����[1X�d��H�S���OX�.���kk�?���j���}�R�S%���Pe���$�
�/������N��C�3�%T�pq��_�P�|S���\Y]��`���\3�i�G���>)����J|�����s��M����8��r�����>,�.��Z��t��Mo:�fC"� ����������O��:��c�n��!�l'��V��~*O,ZN��v���
��Z!���7����o���K&�����w���gL��8��@�0�����5n���|��6�
�JG���<W>1�B�K��_]?��#�����e�~"����SQ-��h���08�y��
�|P�Q�V�'�����$����H��k�������ZF��j>!�`�Cq�d��6>��|ZP>k�����Q|���~��m�]Bn&`9wbrO�>���_��jWT�7ZN�n=����p}Fy]'�<C��cF�T�nV��.�,OEQ��J�/������-��.m�����u��8�P��}3����kX���)7&�.�-�!������w4�������2~�#�+����k����t����h��v�#q��o���C\B�N�Z T�a���� �|���]n/�!�������jQ	�	�*� a�8<��h�����3���i���S�n)eh2�����9&��NMS������:�����������';O�����7�|I��\���|�[\!-��K2���������m�|0�>���k����9}FG�@	�w�I&��\����]�q?��h��O���P'�x����
��������?5$���������<��..#����v	H]��U$���]��/�x��N-���G�C�g�����Z��L,>}�N���[e���~�0��W�@������+�b;)�,m�d����FY��#���g�CjW2�
xCX�-�m�����'��tzO���P��?���<HI$S����5�xg��(�]����/]�YZ;l���u`�['��8�8�@�g���nom'�u
"��Q���1�����5��3��>6�W�go><�i�d������f�j�/���Kf���?�#]��x�>������
�~?�-��� ���'����/
i���=z&���Swd*�Ppa@�~M��>���k��_��z�����b�g���_
x*MJ
oW�u�4����$��Z�
KP�J��~�h�&�C�#�Lj��h=[��Z��W��]C��6����3�n�?P1V�/�N�]���]6�A��f;[����
}z�Ey���>�]�����S�#���}��b����?
��Z��<a>�uk��W�����4w^Ys���� ���=���)������{osr�*��Z����8�:�7�-N�#u{��^��Q��t���x��]_�|qa��+��d��y<�kX����^����H�_�N�/6��6������
�������k��g�4H�Mc-���0�����Ebx��N��� ����+�a����O�'�s\W�ynS�m��	jR�]��l�.KU���N�D)�|�v�~�i��m��,n���������[����{T��G��K���i�j������������$�w�u��F�l�K���o�:�7j�"�y�$r��r��x��X�N~�������Z�=�_�p�~d�A�����W��y��%���I�����s��P���ZV�e%���m<�����}�y/�Id��|U�C+Kakp�' ���B���nt��z���\k:�R��5�m��;x�0h���~
x���->�5�i���Y�m�q��h�����5���h���.-�YZ�3�����*���)Xk����=:�B���Z_.<�3�>�x W��9�v���ZZ��u;����}��bO��zc���m������-���5}6�L�YR��\��;z23@����xH������'���H'?7n+�����O/�"����X�o-�"GPq�Y��0/�m=z�}2�^�����V�ece����0�<#����V�f�o6�������2^����s����z��|o/�'��_�zG��X5�d����g����?�?�2*�>!���|:�.�X��f�XW?.G�4��-�6��mQ�$���^�>E#���H��*���*i�%�E�����u�,�W�2@>���|�������%�/4�,w�s���N�<3�0"�!��W�2�0��4�;����4(�;�9��[��$,��T���
sw��0����o[�q-��g�:��9��J���X���28'Q��������u����(�8-�DEQ��~��@���KJ���m�}Z	�Ym>��E�dc���'�p:������;//��[i0},{A���3Y~+���?�S��^F����)FW{����L���W~�!�@��|��;�$��hY����H�?cy�+���;m:=�"n���s������6~&�e�����W�K[���2q��9�++��oo��^G����2��X)��>�����"(����[�If*F]�`�:�O����2��[]]y�,�k�w �1�k����g��~�����t���r1��S��XxI���	.��d��-b���9����z����/��o�D�����
��!�*��;�=7�:$:��?�m.G#�:��W/�|Y��_�Z����Nn����G�X����|�u����x����V6�r�9�G���QE�����W�u�-l�x���$��jO��M:H������l�n��� g�
ze����a���5��bH�S�)��!��w��^�t�a��#�U�?P@5��8��������������>�Y������.i�F��&��^���d��X�B������t���-t}g����qvH��c��	�p1�����L������U�o����<�
p~�?S^�Tdg���g�t�i"�Z��v�1��L��?�A\K|g��{}�]v�K�������1��3��^��}�����*�^
>K1R������#��w����ch��_�:0���!���@������x5M"�n-g\�=��<Z�?4�cK�^�����*�K����+�09=q��V�7���K��������2�.��O-�����/h�xw��~����,g�1�NMy���x$�Ru�����3^�(2���F��\�Y�����Je��`.�z���������������4�b�D��9;�=H���zv��Yh�\���p�����I����x�����a��B����"����y��q�����=?F�� �{c�j���zb\\�XE��^y�+L]Z-[�Z����}x��8�E-�L����/���v�C�fU���+���dV�b������������t��zR,���9dA�0`����������p�uk����B��2�G��O�E����/�/��j��8��Q�>t��V���
������3�q��3��P�OaPxw�%��<Esc��2��O�m����A�E�,~����B���h|9�mf��
;N�H}K���
�{w<�4�"�C�!���d���p���{��Rh�W�I�������0�������_y�?��X��M�heb%�{�`{��1�?]i��~��O��Er�p$��������=N{�����V�s]Z�C��%�4
�c8�:�5������C���!y9�>�<��27lylg�Q��8<�Y�?�Z�|}xV�Q��g��n
�Py*H��<p����j���<����<���v���&"�5�P{�;�������>S�b��pOL�~U����7�������-���`v���9�Z������E*J��y.��?�����������Y�,:�G'���~�y��9/Y�w�@b=���Y�C�h���E����9��?�]'����D��5h!��F�[���Du��p*A�+��t|���7�^B���`?���>+�j��u!$�6p5���~dd�>�dG�=v_|5�..��<!��s��}����t��5�$8U��9�p��?���.��#}�����J��?��?�?���X��eg��|7�X\�z����f����c���^�x�W����3Ko��u=X�6��J�@����z������
��%�"���R\��k�!���wg���q��@BPI����y������n�Tv2�Ez�����FP�CA ��$�	'�����{a��_�[���������N:���h�f�42���"��F#)� �"���<;�^���ss�L�X� <`�����
q�����mB��ma��y|��Fb��R%�A��{7��U�<3>�sm-�p������N��
N�<1��W^u�^��#�p3v��y�|i�������tr�!=�`Et���}/���oiD6q��y�w3rMbi�9��n�T���%�u�{Wt�����}��G�s����_�u�
�k:N�;�K���>��>�����{9j�K|
Vo�"���c�n��\��w����S���7�o�.GFV���+�x���H�P��rH�#��4?n�K?�|�j���p!*����o�����P�$0X��F������j����<J�K�O��9�]]�#�A�L�PK����g�u	������hAg��\1��;�����?�LS�����$�V�P�;�l����tKm�I+�i��td�;�'��?��h��&)�_���PW���%:���[�E�g�?�$�O�����Z~)��S��%��]f|��M���_���Uv���&���1�b������y/�����#"����-w,�����oH�����]��)�V�<�����:����������U�Z��|cp3
���A���#���������l�?�u��*y�!�4E"�8����\V��~$/����i��u���dh�����q�~���C��"
��0�
���N���L����&���\m�����9�#=�(_�����Y������E�m�{���}�~k��nV��[�oX�i0��5���O���/��?��,����r�J�	� n'<��z����y��O�s��H���x���Y|p�u=	��R����=2�D��\|�W!T�����s�����T�5��@���S��C�t�#����t�x�q~���n.�7��E]�����?�q�����nc�B	]���?_�@���i����u�Y!V��/#������\<5i�P|<���a�gT���w�j|\�~����
$��B� ��& �<g����c����G���%�i,8y��|D�������xo�x��������0o�\.�B�����$��uu+��=��{T�F+�����q�k���C��~�����];�iL����{9���k���(����I'����tncK���a��8��(��w��3����T��ZN�,�W�� u�{���]F����z\���p���.�$~��}���]G�|d�}��tKm��!��d3F[q��?�~<�%���FK]CP��8�~[����.5"��s�:��d�G�c����Ac�=K�������l%�e��0'/�p���k���������%��	h0G
�d��O>����?K���I�5]K���q[D`��W���5�����w^%�4;�����]:��`n<s������K�����������������(_�z����=?R��>�mq�����<ax����~0�[��h����Gy"o��Q�3�px���=}6��;EylJ��#!A��
o�m.����7�*���i$�L|�����`?S@�-�_M����=OO���c�-o�a��}}���x����������?9b��e
��?7����-#���=z��Z����&^��fPG��V�=Nf�K�a����*��4/~/,�2x_�:��%y�#�Q��������o�Z_�m�>������K�7������>���-:�J���ge�v��(�J�O��My������V��K�
�(�+'��Mu�4����>��N�-���\���Z._'�pp:�XK�~��H���MwN�w��yU��9�x�A�.��:xoK]j#l-��V���FTu��=�*����E�PM?��v����h�~�>����z���Z�}e5���x�9�e#~F������^�}���E�aNO��F?1��z�A\��Ex���O�/������������c��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(�%��l��</�������+L�
��(5�Tc4�"�F�r�U_���xoQ��-����X����j���</���������9�Gug#����x�1���^������x���|��-��������D�����i$�����/�_@�\P�k���(�w{c��f�FI��u�i�|�����I��J�5/	�v�`Ek$�f>Z��$�j�Z1�s@O�������/xN���~U�����2|���#����O{p�xg��}�Z���vs���J������j�5^�hn�l���)wF,���#�W�C,t����,�2H�YNOEutP�����_�����w�8HyV&(��V��O��
����u��:��/�g�[.A*3��w�����<���}�������IV��w|A��]7�<a}��������M�J�D��kNI9���?:�p)q@/����-���[-2�S�5�nR�K4/��NrGb	�<u��@����xKA���X��wpl�����f =��l�&�@o�8&��U�Es�����$B�?z�A�WgE��������ZI�4$�Oi��G�^+�-s�>2����,.����� �����O�9�>�_EQ���o.����N�k��!i"�\�V���=����e5_
k�wC�
����$��^�I�@C��J���z����%�t��LP[��Rv|����'O��~�Q@o��������Z[�q'��;!�����������d�la� �zV��?-�.���\I����cgld����th���v:>���s�G���\�C��(������������W��u��l���d'8�N�<�Fq]����5+k?
xOY��eD�{�<��K�q��d��x�����-Z�t/h��|�-��[dR��*�����;h�������������F��x6����:��t�����KL�|W��=��2���$r3�G�����?�<e�i�jZ>���m��������NH�5��P������>������T������-b�Ab�;���~��|����n��� �bc�2�3#v#5����W�|f����������t��.�B�y�����^�F(��h�k�
��w�
�-�r:����(����u��6����A����v�&�a�������g���\mwT��������,���\��$*H�g8�+��I�:P��{����~����\8�P��&r��+��F-"`�^�J�P�|`����w����y�-D�(��2.I���zN�����w�e�]���F���q��b���	��V�Mm/�|a�jpJ�g}c�����3�\���[���2x�J��&�M���/��]����	��8��2s^��z���<�������m#�������R���(<W�Z,�`�k��
���}j���i-�m�ku�!�1E;_�������P�'����Y��'
�2I��V�������]i�� �bRX�0��b��wS�c��[O,QLL�G2���0?��(��1h��\�&���P:-��5���C���t�4����������/���L6��@X�>z����0x��}7P�.�W��0��L�D�%`d�H���Vl,�G�9�]i��i�D�&�N����g��X��?F��\�$i�&
���;��|"���YX�6���	��)P�mg`x<�
z
�����������S���mo,��S>�� �G>���A���<Ak������!>}��ZB'<g98{��@�@a������t�xF�^^���ud>���2=x$��)��s��!.�7��y�5��
�����W�R`u�@��|G�x�O��P��4h��m����&�F9���(�>;��^�V�>�t�9B�;O�]���E�h:��;�Q�����~j���9�n��:�F��F����'��<���>���D���1lhcc�������Z���� �|I���kZN�|7�\2�����\��lc���99��������ie��Ao
�4�!QG@z�?�i1������&�����i`K�D����a�Xz���:��_jvW�U�������k�H��g����k�h�z�����|h�~�M����6_:i����2� mg$��>-�-���jhd�F�=������:�v���_��c�{��z�og�[i�}��aYm�oB3�����^��j��:��7��mu4��+�����8�O^�����+������m��i��i(��p�A\��<\WoEy����H�K�oY�S�B;��oY��.H�>�����Z��MR������M���E���� �����q�8��0)q@�������$Zd�L�q�2�K�#���X���u�b���c����6�����GM�
�wf��O���c0z���~7�wn�x_�:����lI5��V�N	?�Z�lRm�J���
�|/�__k�]OV��� A��$��%�#���u����o���I��I��Z�2��Rp9�y�Fz���@'���w��V����?�	����p�kg������^��/��o
�h������}�,��&�0JZ�����S�������k�@�;�Y7
�����v����
u4���5��9�4����fOPB�q�:����<Q�d2������N1�j��.u?_\h����(��#"G��kF���o��f�O�<'���K�,V�b��N��\���-o�5����hW�D7Y�o�����p	�I�w ��iq@q��Mt�Cqa��4�W�Gb�����I?h�"�:��XA�8 ���m�������[N-�M
��a;!�61�z��'�-��
z("yd{6
��fc��Mu���<�������F���7t�*��
�f�A�v�W��|9�|O��^;������Ab}G<�����<Q���!���={U��ll��Akncfh"W4�)b���u��.o>$Vv�\I����bgll~p5�8����)$�]��n�-�"�,O��sY�����Y��s�H�fRT(����<�uE����-��������9�������eG����`�c������Iom<��y�8����@��k�1E��g���]�g[��-"��G������yC��� �H�k�� ��_<]�|@�}�>�5+}"�l�W7Py�`8�g<����z'�!�/��l2�"J�HS!��9���P�|����k�^[�o'�d;&����9��O�6w7o�o��O?��&&}�)��8Z��1@���W�;{;�=
�X��F[�j	xF8ls�q���}3U��,4��w�w�B�k<!$c��'�z�&y�<��|�tO��TI�cc�'zP��H��{��[�>Ox����wv.����$��Gs�d�*L
��V���#���-{S�n�,��E��h��(��^B�9�
�_�	�����m%���t�h����U�bp>�Wy�(����f��m<_��j�U�l�[��Yw#���]���Kq��&k��{�*19H#�yP*����G�]-x^����	�Y�85
�Z���vn-�l�.�������|U���~j���m���[�����6C���y�k��F(���/>i�f��	�ZQ��(m��zk�����|�O�<S�_^ik3Ig}c�=}�z� ����T`��NM[�O���{m&�M���7�5��liO|\p�2I5�|{����da���w�dDGL�n�
z~(�
Z2���U�R-��>A^icir?i�F��N-�����	��n�3���Q��<K�2
x�{-S�:��pUho��.#�GN�p�:�K�W�m�?�O
jO3\,�yyn#X��N�'=;W���:P+����s0e��D�S��P~��2�/k&��x��S!�^�v��@���T~��%���/y������v�[�b2��29#��������,��=q�B5�c���9?�h�EQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE+��O���t�F�Y?��S�m2�������J�O��������"������zUQ@Q@Q@Q@Q@Q@Q@Q@E�>����P��T�Z��������rs��,�EQEQEQEQEQEQEQEQEQEQEQEQETs�� f���z����^�$(d@���q��`��(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��"�u�yb�,d���j����V0��76~�f�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��"��m�28$d=�?�KU������r�����Y��(��(��(��(��(��(��(��(��(��(��(��(��(��I��$��0��������:�����������EPEPEPEPEPEPEPEPEPEPEPEPEF��a�E>�������{��}�����m�i\��O����x���P_]�7���8�7��P�*����x���G��������	��~�o�=����k������b�&���]������}���{��}�����v���?��Q������(j*����x���G��������	��~�o�=����k������b�&���]������}���{��}�����v���?��Q������(j)�4r�u|u����EPEPEPEPEPEPEP��y����E�c%��Z�����6�#�"����d��k@�EPEPEPEPEPEPEPEPU�$i<��l��������v����w�OEPEPEPEPEPEPEPEPEPEPEPEPEPU�dh�B��`��a�X�.��%�3��\��<��=Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@�����
�n\��������q�g��|�����S�EPEPEPEPEPEPEPEPEPEPEPEPEP{�;R����#?�*�Axe�����zzn�*z(��(��(��(��(��(��(��(��(��+3Y�[�8��mln��E���c�mV��Zu��
b�v�V� CgJ���������9�x�_�ou�It[9f�6F�����f�nu`����<U�i������H���T������3�)m���pFzS<'^
�s���)�	dW��)������B����oJ���u�cX�I��h�:�#Ia���V���:��7�E��gC�,!]���`��c�a�Hq�S����k��&x�E��C���l����l���CG�w��c�u�x�F�O�����K]B�-�f��Ids�PW=O�@�[�_�F��2�����������)f$��U��1�W<9�����7Ux�-{k�0��nPq��k������}wW[=&����������
�|(������p��l�1{��o�}�+x���@a��������0����:������P�@��=��A������6zg��J��+7Y���"����
G�b�lcW)�rw2�@TW3�x��T�����Y����\%���f�3����0��-������g5������Q!�*@�=|PAEsz�� �.�b������?����-p	��[�����][M��N�g���I�����4W?���4��mc�����CK%���\���F��������-I�R����g\�����AA�(�Q@Q@Q@Q@Q@Q@Q@Cv�;I]x*�����/�����W���k8��WfP��2I5?�-?��?�����<`��k���o����#�Z��mN��^�=lDh��������}���i�cj�o����	��?_�
���H�O����k���'����?��v_`���x���>�i�>���5���E����C_���-S�~����T�t��?��v_`���x���>�k�>���5���J��K�w/�.��g>�J��y��\���QU��+����}���h�
��������^	�5_x~�S��> ��]B���������{f���_�Z�����`���x�����|J�_���Y����M<�-�2y�&��q�\�w���>4�|B��g�7�#70�@A#�=s��
��Z�����`���x������F�5=CR��t}r�ZH`�,%�YRR�^�H�5�k4�+X}&��P���4��e�������9��������G�-��?�����/�"���j��ue�m�����0����������}^��K}J�;�<�;��6����c�=�h�������|��_�������x�M������wD�{;HL�K��(�=NG��i�#����+�+�e-��)�OF�B=�4��_������������\�����U����R����^^ZY��[7p�=;�UO�w_l����v�X�1�\�W�;pOlt�����-^��km� ��Q�?�����
�@Q@Q@Q@Q@Q@Q@Q@z~5���^������k�OO��������"������zUQ@Q@Q@Q@Q@Q@Q@Q@W�GO;�l�V#��g��U��PM���g'�zP�(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
�x�� ����q�y�Z��@���r{����EPEPEPEPEPEPEPEPEPEPEPEPEPEPxQ�����[n����x�ZU��`���eGU���EPEPEPEPEPEPEPEPEPEPEPEPEP{�y-J�v��9�?�f�Uk�Y-
��
�w��Y��(��(��(��(��(��(��(��(�����	eD$d`*J�<\�!���?�1�iS���0�Ut����}���~"�����l�R�M)�����AF���������/�=�l�B�4�`��A�Pi��t�"�9lW��(��}Mw�)�26�+����z�Ri�k��Kp�r��p�y����Q�5�?�e����R7�z���n�B���������Q��(����2�S���k�?��Q�7�y<�*�Tk��3��y������D7gp�u}8�YU��q��������h�j��l,'��`�[�����UFI��M��Uu=:�X�g��(�{[��,LH���r���6����u��+}�\6�(#��?������.���?����[WV��5��(�A����c�s�?|'�j^��)�.��	�$��s�zOk�_h��,���y.�1��x3H{��X�V���YxO�z~��Cl�<����$�*���<1���7Zj����U�E203��aW���v�����W���'������r�I� ���
>(�$�k��h��z�O���������E�O�xK����ik1�x�I4�c�����;�x�����|7����w�r�{2��0�Df
0���j�z]��`�Zl~M�U�`���h�����QEQEQEQEQEQE��x��\������������@��x��\��W�����x�:�K/���������@�	$�&O����V4�s���kr�8J+���K��4����[@����c�'�k��p��/�����Q]����>�������K��4������va��S�1�������f���i�Z��6��e���f���NnH����*j+��x&��^�&����x��.�{[K��C��z�Z��h�~�h��M�[B����I���'����P�N���/�^�#�����^rD�I)����#=X���P���>:�Z?�.5�"�
�q2�60�������^��xWD�<q&���z!9�����"�M������w������nm��]���b������s_
]V��Q"����� ��W�gqoi�x�N��#���4����`��=Wo��^x�w��k7ZD
�����nV,�	�������/��%�&���{��YYHp=7=�������	���;�[W�F(����q�T������hT�t����V����-�����[��En����n
���j���i�������s
��b@�*x=�y�����xQ��y��vw)N%+!/2�98��'�rZ��
SQ�����
����@���$ ��<�W�h:W�,���XA{w�3����qU4/�{�R���mi,���E%����I��r_	/�,~���[\�\�
Efp��a$�}F9�~
<2x��m�j�M�B�Q��+���/�5=dj��%����et��t,:7�
iiZ=��m%��l��K3����w9f���j��g�]�^�:��|Y��aW���(��(��(��(������nY�9��;O��TGo>�%e8���95�����a��g�=G�6:��6�l�
=b��jg�I*~��v��+;9�n\G(�H���'���-e�?� ���q�^���9Kx�l��4�����^8��^[H�m�����/�g����?1�E������m�!����}F�N���1��~�n���������hO���y�s�f8����������7�a�v�s�P��|e��x�K������\���i��b��z���p=����<U����J�|{5��c�1��R���b�tG^�<`��j���Q�y��;�������K���/��?��!��q�?5z�����|��{�_�2^�5�I����|3���X���V�B������[�$���Q��1���QY�b��c�����B<������I����A{����0�����EV������wu��q�������
��%[���,�U��_)�v�={��s��|����S�\��'����������)��zD�1�'.�n����*��Knh<���+c��E���h<��v�1���*���������V�}��qw��-�	�w)��z�E@��9h|���wc��R ���c@Wn�9�l�@(��/|���o���l~<���$��y��;[n?>�f�Y����3�>��`��^yI���9�J�=��W��Cy�f�S0�x'-�H��@4T.��6�>VFAS��.�������
w{w�	�����?����������(���sA�dm![���,�U���F3<���~t8���[A�>�lg���EWqw�-�	���s���_h���s�i���q@�U�^oo1�+��j���y�A{����~>L+c>���*�/<��������q�������Z3���c���*������������sNau������2
�����'��Qw�����S���61y�����[��4f�����m�����l{����F3w'kc�Z�EVq{�������[�������[@;����X�����>�>Nzm;���)^oo1�+��j���z�EVA{�����a[���^�'-��������@h��/<���y�;�V��9��������r������z#0�����Lm�p�9���64>WN�~�Z��VSq�<j�T���9�
*�b�����v�u���
/|���y�HV��y�4Ub/|����������C��������*���4f�����_-���r���z\]}��C�����\��'������h
`�j��w�A{����v>M����4f����r�y���lc������Z3'q*���4f�� ��yM�~m��}��Nqw������T���@�P(��A���Y8N�n������h.�=}��Ue�S�h<�6���|�F/<��������q���4Ug�Zlh7����3��+�����p7nS���@(�1u������N�}s�D~a���j��w�Ud�{A�&�lg����NZ3<��~tf��E���Z3'q*��lsC��'����������EWqw�
���AS���*����������8�	�������h�v�V���J^�O�����#c�<����%����8�Y��~�g�	������m�c���/|���o���l~��*���z�m\
���=��)_h���s�i���q@�U�]���h|�pN}�� ���c@[.�lg���EV��9h<��B�1��E���Z3'q*���4f����	�����+u������64>WN�~�b��������S�\��1y��0v�S��y�Ue�S�h7��
���4b����y���m�����*���-64�w�[�4�����`n���|s@(���G
���N�}s�D~i�/�S�n�b����k���|�Q��~h��l�f~R����,�U���W
����lc�������q��V���Y���������� ���zP.���4>Nz;���(z*�b�sy�;v���4�/|���o�f���h�[�O�����km�����Zlh7��*��9��Fm����yQ����Y��G���Z<��A�w���D]}��h|��*wc��P�Ut~q���|�9��H��k��8�v�u���,�U���[-����lc����%p�y���lc��4Ug�S�h���[��4�.����>_�>���_h���s�)���qI���k@F��z��4b�����}��6a[�4b����y���m�����*��-64���l{c�Y�W�h�7nS��9�T]}��h|��*wc��R �������)����,QU�^�1�'o��[��=(��m�����lc���\���)�q���t�^�#
��������_��-�������;�t���q�U���SN���}(��v�z�<y���7w��@���~T\|�c�������6;g�E���u���?*>m�������Y�����v�>l��}+:��r���^ne����xu[0�����R�������|����y���������sE����(��w�)�l��sy	E/�q���o0�������
�����A�q���������_fh���>b�~����2��<���-;~giE@���kC�dd;������;�+��w�z���'��F/0�k@N��j�_~zP���m�����lc�y�4Ub/|����������C��������V�}��4Uw�`�����>��qu������N�}s�������h
����9������cA�&�lg���EV���r�y��;[n?>�0���kA�s��l{c��P^�������*k����h6�=}������6����r
��9�8�	������?�OU��}���;{W�<��-8$}���z��V4�#�
������z
6Ll9�W��{��O�������F0Z�
��V_Z�u�db�/��:+�?�=�?�������U���������o�*��S��We���Es~���5��$(!
G��g$���]��v��[�jq�G�Z��Tt��E4o����C����x��7Wz��4��7+����Q������h�7L��;����4��v�C�Wrw�MY����M7R{�v�;��d��z3�UO���h��S]��m7�GS����%@���'��OlU���-��h����0�h����${��(�a��.�<0�.���k$��������T�|W=����|Q�#�����4�Y��7�ff����w�S~}����~6���)�~x��Q�n���S�?�����
�Y���>�����������
��6�c>���(�=h��(��(��(�7����5��i^��E��[w>j���O��b;c����xw�vE�+t�p1�O�r���<��>�����m�x-��f����q/�.}�W<�A�Qh��?�~!Zx��V�\:T-��C8��o�#��>�{zs��P�x��:���K��������[�gr�����d���Y�xs��8������M��4�Cqoa���y�32���&�F�C���6�#�"����d��k^�z~5���^������h����(��(��(��(��(��(��(��*�������fpv������{W���4c�^1�x�QEQEQEQEQEQEQEQEQEQEQEQEQEZ��p�dW��0�����U�D�LC$����3�X��(��(��(��(��(��(��(��(��(��(��(��(��(��(���B*���[��f�����q�]�:��=��QEQEQEQEQEQEQEQEQEQEQEQEQEV�q�fE�nQ��}�Vj������}��3�����PEPEPEPEPEPEPEPEP\���)�q���u��x��B������N��;�,���+�<��(��(��(��G�r�ZU����O�k����������}�T��+���B�(��(��(������
�����A��y���A�����"�p��G�������mQ^���EPEPEPEPEPP^�������*���������TY����/����s���{��x��\��V�.��I79'?������*��=��K
��:��<���O�A4��\�����#xI�nO����W'�*�����<���O�A4�[�������M'��?����G�*����3>������6������EyZ��L��=3��Z��FQg�c�B�"U!�����
�!�������Hu�?�&�K��^�Q�!�n���c��xn���l�Zi��-��
�����O8����<�L��t�^8�������B����o6R�b>l����<W{��NM"�����Z���������!�����zM���m|U}�������3�6�Lpk��~��s�aq�iq�9y�����)�;>��yS��|7����U�����k,����s���u��vX|C�\�!�t��%�8�����!����S���+���.��xb�A��Agh���fa,.3�W�wry��6��;}B�P�]B����\%���K#�%���'�g�c���7���Z��+X�7�R�����f��Z������c��4Scj��`$$��w<�Mu �>��-F
Fv���!O-o,.	Jt��}h���to������sC.�K�����X�I$�@Q@Q@u�������UC�>,�����QEQEQEQEQEQEQE���^m�G�E���{���$��k������/�/���QEQEQEQEQEQEQEQET�Q����V+��=�z�g����n���9�OJ�EPEPEPEPEPEPEPEPEPEPEPEPEPP]�Z%�3��Rpq�y�*z�{��1*��p���EPEPEPEPEPEPEPEPEPEPEPEPEPEP�E���>Y����s���Z�����;�������QEQEQEQEQEQEQEQEQEQEQEQEQEAx%kb-�r�8�p��S�{���R��>e9c��
�@Q@Q@Q@Q@Q@Q@Q@Q@r-���?���f5��!���
C�\�c]8_�X���
�(�T�B�(��(��(��u����iVn���?��73�v#�xo�F1��R�(��?Q
(��
(��
(��
�>��6�����W]������q������^m��/��u�QEz��Q@Q@Q@Q@Q@Cv���U��?*���cs	��y�
���}������������<X�8��f����~T7�!����}
k�S�T���������t���2����q�v8�������B��C�=S���m�����G�m�����@}����������T���C��g�>��Q��g�>��P�h��z���(�D?��?��P�f����~Tf����~T7�!����}
>���O��T?������������M�����B��C�=S���m�����G�m�����@}����������T���C��g�>��Q��g�>��P�h��z���(�D?��?��P�f����~Tf����~T7�!����}
>���O��T?������������Cw"Myh�2�	7�8�*(m`��J�� T�QEQEQEQEQEQEQE���^m�G�E���{���$��k������/�/���QEQEQEQEQEQEQEQEUk5E�������1��EY��~_������v�\��P�(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
�z�����58�N��Vj�������jco�����,�EQEQEQEQEQEQEQEQEQEQEQEQEQEZA}r����n\}�8�5Z/��;7o�7��N1Vh��(��(��(��(��(��(��(��(��(��(��(��(��(�����+�k�~lg��*�V����|��7/�����Y��(��(��(��(��(��(��(��(����R��������o��!��?�1��/�,w�Y�EW�x!EPEPEPY�����v��7Q�\�������?��<9�#�?��EW����Q@Q@Q@w���y�]���+�������o?����Et���/6�u���:�(��s��(��(��(��(��(��(��(��j
�c"��Ze��r*�V�<�������vu�(�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@z~5���^������k�OO��������"������zUQ@Q@Q@Q@Q@Q@Q@Q@V�do;bl��9��Z�U��2y�U]�2�g�X��(��(��(��(��(��(��(��(��(��(��(��(��(����
x�P�`�5f���c�
��dA�3����(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��+@�o�USk.�����Y���Z��
��'"�PEPEPEPEPEPEPEPEPEPEPEPEPEPk�D�&D�r����
�U��1��UW;�a��U�(��(��(��(��(��(��(��(��+��o��!��?�1�����R����������TQEz��QEQEQE����O�kJ�u����y�����C���1�����QE|���QEQEQEWy���A�����"�:�>��6�����WN����o�Y|�3���+�>0(��(��(��(��(��(��(��*����24��(�W8�"��{�VR:���XdEX��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(O��������"������zI����|��{�_�2^�5�J��(��(��(��(��(��(��(��(�-L��������x����H�o�s�X����@(��(��(��(��(��(��(��(��(��(��(��(��(��*�(�|���8��J���$���	���y�QEQEQEQEQEQEQEQEQEQEQEQEQEQEA	���$��6�yt������yp�r��`��q���QEQEQEQEQEQEQEQEQEQEQEQEQEAxe���~��;n�3S�{��Kb���Nwc��?�X��(��(��(��(��(��(��(��(����R��������o��!��?�1��/�,w�Y�EW�x!EPEPEPY�����v��7Q�\�������?��<7�#�?��EW����Q@Q@Q@w���y�]���+�������o?����Et���/6�u���:�(��s��(��(��(��(��(��(��(���2�G6��8���QS�{��K7XN�8�����X��(��(��(��(��(��(��(��(��(��(��(��(��C@'�<a����<!��-����7�]d�e8��r�OA�5?����:�/���M�t�
�D�D�q��8<��y��������������k%�Mw%�l��UM���,y#��{����?������w���`����3�rmtc���{����x���qx+��kmz��k�5����C�T'v<u�����x{���|w=����it�N�'{/����>���������<{���-v���q�)~+��F�k�����~�v�s�P��|a���}/��Ko��F6�K����������{{�u���]��i:?��,�m;[s��oo�43�GQ�F���lq�
Q�y��.�����.q�����A~�����o's�P����y����E�c%��Z����x��;_Oa������Y��Ax;�)%r��N�u�8��{�����Gc
���\������L���j���}��_��F@^h�[m���63w_,�L��y��Ia��c8>���,�U�[�W��07n���������Y"�|��}s@�U�n������A��4���?�,$���>���*�[�%�K������f���J�,>fy>Y�>��4Ug[����l|��H'��W[�(1�N2
}���*�_h��/'?wa���io7�$%pv�B0{w�Ue[�-��	||�Fp>�����#�������1��Y��h�����rv�����R���&3�����n���4�����3���9�@4T�_hd�E��(wc��n������A��4=Z5���a'o��20}��V��`��d��Dg{���*�[�$,>fy>Y�>�����yr�11�	���,�U�n�����	>��v�}�&H�����v>��	����a����;B���oo�XK��"2��@h��o<�<�|��,�L��y� Ya��c8>���,�U�[��������''��s-��Y"deJ���hz*[�<��#8}��65���!<|�c#���EVU���4��26�����o<�����>Y�>��4Ug[�yr�91�	�������\����(I��@(�6�}�>d^N~���\�"��o2HJ�� ���X��"�yo�XKc�"2>���� ������c���EVe��P,��2w������g�$#���FNO�4f�_ ����[�qNe���I"deJ����]����M��R�c;������Z5���!?/��20}��V��p��_#i�\���*���$,>f�O�q��hu�(�%�6>bc$��Y���������BI=�����F|������v>��	����bF�$�����"-�G�%��>R# ��Y����H2��g���c��o<�,"L���p}03@h��-�	���~m�����9���$�E��(s��hz*[���Db���;���$kx7y�Bx�v�F�4b���y�8ia2q����Fh�y��$������1��Y��:��d����Ld�}��u�.�\����	$��@(�6�}�"H�����v>��E�2HJ`� �w�Ud[���XKc�"2>���� ����q��h�X����e����Lgv����'�, ���9>���*�����!	>���V����r~P�v>��	��������	�N������
��S���m"3���h��\�ub�2������t)q��l�#pYw�3�q�j����6K||��H?Nh�^E�.�\����	$��JV��E����;���OEWE��$���!��"-�W�%��>R##���EVy��e����8��4��T,���;�����h�Y�������6���������$�G�ABO�9�Tn����/'?t!���i#[�[��0v������,QU�o<�,%��Dg�<����������q��z��*�����%�>�c$�4�-�e���
��''�z�E@V��E����;���"-����8>���,QU�o��,%��b20}��-��������g}3@h��o<�X|����pG�4:�O.XA��������,�U�n�������(I��4�n����/'?t!���hz*�kx��!#n������V��}��_��Fp=s�Y��m��q���n���c��o<��,!��Lg���$�*��7)��>��5����n7�G����Im��=3VJ�}�+$^N~�C�\��Ut[�8��8>����+���N>]�����Y����Xa�26��L�V��P%���$�q��h�Y���yr�>l�NO�4��f`c�	>���,QP��FL�y9���}sI��o2HH���20{w�Ue[�-��	~6���F��'l>f��Y�>��4Uf[�-K~w����4�-�+�IF�������*�_h�����P���4��Bb^H�|� �s��EVE�
�d�����>����`���dm"3�>��5�x��B������LV��K�����c���|V%���R�@����p��8���Lj)�6x�>�|��c�^���c���������y�Qq������c�~�|��3�����:�o������=����������'����g�>����M������\3�G����aG�h��=>�q��Z�ky���>�J)F;���1�A���y�3�%q���8��J�a��	E/�����8����s>�J�>��6�����W	�z}k��2�>�����2	?tz��/�#��]�����v�T�_hd�E��(wc��n��^H�\�9��5�OEV�o2HI�����~hU��X4��26���3@h��o<�����>Y�>�����l������O�4f���vd\����(I��K����"�s�v����'����a����;B���oo�K	l|�F@���EV�y��e������1����R��';����`f�,�U�[�������2r}��2�}��!FT����hz*[�<��#N!���i�����I	���>����� ��EgX����"�[�)�K	�#i�\��{����'�%9�FH�F8��h�U�o
'�,!��� �nid[�"�rB!��}{��*
�_h���������4��bF�$��B�z�EVE�����>R# ��o<�����>Y�>��4Uf[�)��'q1�L�"��\�����9>���*[�<�!FT���9�V��%�����w{s�����
�d�����>���y�8ia2q����Fh�[m��������c��oi�XCc�&2A�s@h���e�����!$���v�}�>d^N~���\��Ut[�#y�BSP�����#���[!����,�@�9�EQEQEQEy��<!���k_�Z���nmo,n��]��
�����i[xc��3��������H��9��u��{�7gv0?/s^�Ey���'�E�_%�����d�4��<��A�X?f}�C��W��!��k���i���E�i��y�����	����>�Ey��|����/�{o��:6�K[���
��d���V}��|]�_i:��l���7Ds=��oq������8``��P����k������/�/������y����E�c%��Z��(��
(��
(��
(��
(��
(��
(��
(��
�fc>w��|��s��MY��n����fpq���f�(��(��(��(��(��(��(��(��(��(��(��(��(��_�)��a��0q����U��$(Y�*7l���,�EQEQEQEQEQEQEQEQEQEQEQEQEQEZ���8�������U�p��*T�����w�4QEQEQEQEQEQEQEQEQEQEQEQEQEZ��-	�K.��s�c��5Z��v����[��Vh��(��(��(��(��(��(��(��(�C������������<[�!H����k��C��aQE��QEQEQEVn���?��*���'�������}��>��*QE�g�!EPEPEP]������q�������� ����?�]8_������e�����(�\����(��(��(��(��(��(��(����ld3)d� wf�j#���@1��C��4QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE!����|��{�_�2^�5�I=?�o�?�/x���K����TQEQEQEQEQEQEQEQE^��;�����Fx���y�~��L���=Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@W�y$1�"��������Jc_#;���>\��P�QEQEQEQEQEQEQEQEQEQEQEQEQEQEW��k��a�]�N�g�y�V*D�����,���x���S�EPEPEPEPEPEPEPEPEPEPEPEPEP{��-I�e�/s�C?�X�/���|���8�3�f���(��(��(��(��(��(��(��(����R��������o��!��?�1��/�,w�Y�EW�x!EPEPEPY�����v��7Q�\�������?��<7�#�?��EW����Q@Q@Q@w���y�]���+�������o?����Et���/6�u���:�(��s��(��(��(��(��(��(��(���<���B78�����Az&6�-��q���	���(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��C���6�#�"����d��k^�z~5���^������h����(��(��(��(��(��(��(��*��f?;s�n��`�=*�V�T_;c���O��J�EPEPEPEPEPEPEPEPEPEPEPEPEPU�P�e\H�$��t�Z�Q�A#�jq����,�EQEQEQEQEQEQEQEQEQEQEQEQEQEZ����e!��A�p*�V�P_\���m��w��@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@��2Z�VT;���?�U��~���H�r����
�@Q@Q@Q@Q@Q@Q@Q@Q@r-���?���f5��!���
C�\�c]8_�X���
�(�T�B�(��(��(��u����iVn���?��73�v#�xs�F1��R�(��?Q
(��
(��
(��
�>��6�����W]������q������^m��/��u�QEz��Q@Q@Q@Q@Q@Q@Q@W���e"+*����w�[PT{G���-���@h��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��C���6�#�"����d��k^�z~5���^������h����(��(��(��(��(��(��(��*�����+w�����<�f�Y�7����L�����h�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@V������c�Lm��1����z����7�5���@h��(��(��(��(��(��(��(��(��(��(��(��(��(���������������5ZC}r��]��?{�*�QEQEQEQEQEQEQEQEQEQEQEQEQEV����|��7/�����Y����jL��.��s���4QEQEQEQEQEQEQEQEW!���
C�\�c]}r-���?���f5���!���0����O(��(��(��+7Q�\����f�?�����s?�g�>��?�cG�(���3���(��(��(������o?����Epu�|?��m��w���/�Q������~g[EW�|`QEQEQEQEQEQEQEUmC���~��go^���mA�,dic�c+�g�@h��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��C���6�#�"����d��k^�z~5���^������h����(��(��(��(��(��(��(��*���O;(��f_�c8=j�Aj����g�X.F>\�@�EQEQEQEQEQEQEQEQEQEQEQEQEU{�p���a�q������X����1A�����	���(��(��(��(��(��(��(��(��(��(��(��(��(�����nSb��p�rr;������	3���2=��jz(��(��(��(��(��(��(��(��(��(��(��(��(���}!���E�Fdr�U����[bm��r��p��S�EPEPEPEPEPEPEPEP\���)�q���u��x��B������N��;�,���+�<��(��(��(��G�r�ZU����O�k����������}�T��+���B�(��(��(������
�����A��y���A�����"�p��G�������mQ^���EPEPEPEPEPEPEPU��1YH����� �*�Az����|�8�{���(��(��(��(��(��(��(��(��(��(��(��(��(�,�F�����0�_��#5�i_kbl��,���};����:���Q����Y�1[q�E#
�F}x��~%�]����_�����i0��l.<��NX����s6O�|7���xvMn�X�<@��	'�-��3��#���r���x�S��~	�<��W0�}�Q��"�E����������_���_�<'���mR-V�N���@K����q�L{z�S�|~��=��-J���q�)~+s����o���:�����
_��[��z_�|�x�}B6����7���[{���?�������M#I���k��z����P��[����x ��P��~�������9����~�����*#����o's�P����y����E�c%��Z����x��=?�7V ��e����h������;����@�Eg��j�X���������C�Y�����S�w�7<E�6�}Fy�4Uo.�������yG��hh��hx���&"A�3�Y��I�+�����NO�Nd�7�h�Y)��\��UtK�1/4f>p�2�9�H��2x����"0>h�XGy��i�2dmo(��h1�y*�y�����>��4Ug���ys���LD��s�+�����S��$����E@�����Y��Y�>��D��d�2��20{w�Ue��F����)�g�<��$�>/3w���1�����h�'����V+�x�����q��b8>�g��o���M�'2����y��h�P2\��Vh�Y)�����Br^h�\�Ddns@�U����'��_�����;�)�OrF��4f��c��@��g����f����l� �|��NO�<P�*����JML�$���]�?h�f������q��OEW�.��d�6\�FFn������b>R" �<��*��w�����n����}3CGvb@��gs���Y��I�����~l�NO�Nd�7����2�3�w�4=]R�NK���(����i#��o�'��>\DF��Y��,w~K�#!#k���w�Hx�����pG�4f����@���������Y���&�W �I>�����x����,Z5���J���9�,�`8^pT�wr:Tx������E��&���,��w�iZ!�*!G���x�����|c|t��&�+�G�$��pcHP wy���REZ�<A�j���k[Zj�\�����+�pz�=
uTW=�]r�A��|�K�B��[[[����n���P$������5����FXv��XI$�l��1�PEU�G�DGH���3�+%���1S'��V����|��V/5H?{����Z8���2x�+��20}z�Y~"�/|?�{�Ka���=�C��������:���&�����x�Tx���# �N0>���j><���}=���D���A��I����Ux��+�4�cO����$�]������]>�:�f�`R���1���iZ������xN��
:K���;�7<����"���z��K��t�7I���1�0R�n�����x��sQ��#�u�;�6I����Y�������=J�	��P��g��,�e�Q���|�H@�F@gv��95��:��o�������	
�^�t-�rf*Q��g<�"�:�WV��4�/�[�����Rp�I����/����J��$� M�<RG������5�x�n�<�r������L�'1��~l~4���|k���M
o=�<	���3�%���{EV�y$��3�yG�f�����g�8's����EV�;���%���������tg&�E��1�}��OE@�t.	i�1d�Dg?�i��v7��D�.##��4Ue��D�������4����y��o(���Y��<w�4	<A�����~��J�v]|��U��d���@(�6]}�w���������DK�!2M'82��4b����p��X��������w~Ix����Q�L��*����P��$�o(���hx��O.x���f"r}���,�U�.���������>���R�\i�1d��3�}s@�U���n�'���n"#�����Os��D@����o.�Q���8���Y��r&�+s	��wb3���j��xc@������3�Y����e���5\
��NO~��.���M������������t%%���8Qzs�D��+����G�DD`�|��*���� ����8��w�J��!&N����f�,�U�;���%!~l�NO��+����1S$����E@���h�Y��Y�>���.�o2h�v�20:�EVX��Nx��ma~��.�������yG��h�Y��1�I��LD����H��/�4J079=���*�?h�&������q��"%����2��Dd��h�Y#�
�d�1#�""0}����������8���EV1�y*�	2w7�pG�4<weS��% |��NO�<P�*�����1S'������F��3~��s��hz*�q��o6h�v�20{w�X�nx��l""���4f�����8���7}�(���
��Oq���H?A�(oRG�"����n�*�P�!��>�i���'p���R���h�9��Y�>��	����Bb^h�|�Dd��i;���O>R"#���,�U�w�K<FL�7�p�4�<�x������>��4Ug���ys���������t�3�F#�*c$�^s@(�\��q�?+?s�9��4���f�&��h����X��$wb7<E�6�}Fy����q���n��Q�=1��EVh��Hx���&"A���I�+����vc''�z�E@R���X�Y��Y�>��D��Fc�
# �s��EVH���<LJ��DF�<�#��X4�26��p�4f���R������w�Hx����Q�L�+�� �-����H\������D���Cg�1��
�����K��q�S@n�~T�9#���C�]�QM�c����6:����*�:�i
����6��J.+.��7Q�\����
�H������&�x�^ne���Q�<:��G��EZ)x�C��g����Z�g�����?�J)F;���c�:-����?�(���~�����[�9�a(����tq��Zv�g���%w���y�]���+��=>��x.O��<����!�w��q]e��y��o-;~giE@�rnY�deLg?�i.���������>���c��Uh�����bW��D`��y�c��X4�	XDp?��*����O�����1���D�<A������nx�4UwK�"���S �I>���e��7y��Y��Y�>��	�����v2M.��n������1!����EV�w��g����yG�f�K�)B�q��b$�4f��"]��\�.������Y.M�+4b,��������*������3NFA��4�����Od|������U��G�u���0s���"�v"`��\���G��A{
����\�����dd{��Y��(�9�V�&"r}���D�2)�h�p2d����,Q^m�OkM�K/�K��H�����u�I����'< ������<I��OI�j�&O��j���t�K���w�
���rs��Z+������I��+������,�d	��\��!8�v�����?O
��K�E=�^��D�����`�8���h���u]V�L0���VwL���y�F{)8�3��Aw�
�zF�.�����(���d���"A�������@�%���1S'����r.	i�1d�Dg?�hz*�q���d�6W��D`������D������^3�Y��yw�H|^f�[�8��3C�xc@�������f���v]LsF��������e��7y��Y��Y�>��	����BBd�6Lzs�D��#��"�|�DF�<��(�=h��(��(��(�#_���a�\6�������L+5�A4g0'�����'�<{i��e�tM��X�t���;9�q����tP�������!���M�����-������1�Ha����b�|%�]w���/���Z6�E�i�rV"�zGs��c��zM��������4�x6kh��5��"+�[�:N�lVu���[�iZ��`��ltW3Zi�s������o�z}���^m�G�E���{���$��k������/�/���QEQEQEQEQEQEQEQEUk1���-��-���*�V�1�;�V_�>���<��EPEPEPEPEPEPEPEPE��mM;��FY�j7i+�J��=�����M�+[�����2@c�	npq��K�����~G�(���?��?#�~�}�������<�,�G��D���������x��	�(!7����t�9���E|5uq��<l^���e����%�PcE���N���_��_�������{K����#�{9�a��2��
�}GO���(�<O����Rq�{����p>�����������"k^&�%��t����
B���#����3�]_����?�����K������Q���������8_���Z��I�s�C|�7^Z8cl�U������:��������%���5�?�
����i��/h���wg��v���X�v���
����i������mK���iUk��
	X��S��`U��3��?�Y~"�����R���
l$r9�)BqM���:u&���v:Z+�?�������������������o��?���}�$��}��\���Z���/�����9
������{���-����{�U�m��Z^�W,x%�N	#��5��i���u���h���?������������I�:��F�<-��:-��%��ofP����on�0�����n��M7E[���D�7�����Y�`��<^w��������o���OP��������G�#�?�'���=��!�'o����k+eQ�V2�S�p�C�j]��~�=�������
�]�N�J��uw7E<
���=C�������z�������>����?�_q��W�i���u���kC@�/e�
�Iyp���U�b�i��J�O&� ����z}�Wi�Q@Q@Q@Q@Q@Q@Q@Q@�����K���8�5Z��B+7�x<q��@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@��mhD�Qw/ g���Y���1j|�f]�����j�QEQEQEQEQEQEQEQE�x��B������_\���)�q���t��qc���*(��S�
(��
(��
(��
���'���Y�����v��������������EJ(��L�D(��(��(��+��� ����?�\w���y�]���+��Tyy����_���E��QEQEQE�����I'
Z�x#���	�<R�GS��#��*��!��%M9u�-t��*%�-�v�s�#?�]?�to[i��5�&����nr����^r@�j������I�&��ZZ���8f�
�r3��J�����b�t�t�+�����L��q�m�I�a@��.������kh�W���q��s4��\��#��/���x�]+�����=���J�1+�u$d28<�[����#�
K4��b3F��|� �����b�������vm5����Z���9���f;@��	>#�������b�#IB�1�uM�w#vq�\�����x�J7��4������WW~s\9L��v��
���9{�
k?���d��O��l�0��b�z���z���!�um"o_��k����R�'V��)Ug�p�Ny4�
����A3N2@���:U=VX�����Y�E�����]��Fky"Y"�T:u\�d{����`X�����;�4�����;��w�k�R�w�gb������z����Xi �.��u(G�[��rKM���	'���A�K�=SB�����^x����{y���:g���J�b�?�?���;��w�h���������?���R�����1���G����Y�oq�]_�5�����$o�1�_�B��U�!��T�������k�WP�48��+��3�P0����������9�4�`?�����>�K�gb���w��xV�������s�6��b�vd�)�>�SX^�E�����������jWV��H��0��c���kO�������?���O�?�����>�K�gb���(�O��������R�x������4�*v�q�O���'�������E��QEQEQEQEQ\����u��l�!sc��6���(dx@��}	9nO��3��
=I�(ua� �Q^c�
;L���G���j����xcC�E������Sy��N���/�����(�G�@�Es�_�i=��^��hd�0P\&>�!��WA@Q@Q@Q@z~5���^������k�OO��������"������zUQ@Q@Q@Q@Q@Q@Q@Q@V�u;j*bg�z���h��w�1�X1��(�Q@Q@Q@Q@Q@Q@Q@Q@gk����s������@W_�QWO�^�U��/Fp�QE{g��Q@Q@Q@Aw��}*z���=d�V5��/G�X/���_�2h���s����(��(��(�/���a�]��5�Z^�������j��k����?��q�Q@�(�t��(��(��(��(��(��(��(��(����_\�ER�2��q���x]���XaWn����{��(��(��(��(��(��(��(��(��(��(��(��(��(���~��gE�nQ���f����jZ!��(�3�C5b�
(��
(��
(��
(��
(��
(��
(��
(��
�<[�!H����k��C����������p��8���fQ^���Q@Q@Q@f�?�������G�r�^ng���G�����c��"�Q_&~�QEQEQE�|?��m��w�������
�����A���*<����_/��h�����
(��
(��
l�V6a�i����?���P��9!����m�u����K/��������XK�G����cO���Z�����%�������o�������o�|���{}��U��������x���-�4�Y}�<m�������cO�}j�����_�������K/��������XtQ�i��V�cs�����o�|���}C��wZ|�K
��#*p�#=��]G?�{�����jN��15�X�.��1�J(��(��(��(��(��*���!�����BR��?�������!U���%�z�Q^���QEQEQEQEPx�(����u
S��5�����v�v���Q���s�8����::�����|T_H�kpf�D1%��1�����v�(�[y��������h6��j���@�,�w)$m$f�|G��mO��
��(,,�D� �Z����9��im0���e p��IL�5��?��~�S���(��(��(O��������"������zI����|��{�_�2^�5�J��(��(��(��(��(��(��(��(�-D��������������q�y��6���p3��,QEQEQEQEQEQEQEQEVv�� +��?������u���t�5�e[�R�g	EW�|�QEQEQET��'���.���O�c_�R�����z��%��&�(��?h
(��
(��
(��
��������#Y�����l?���F����Lq�����W���O���(��(��(��(��(��(��(��(�!}�������y�������F�{r��
�Gf�
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(�� ��c�|���=7��=W���jUX)��,q�B�PEPEPEPEPEPEPEPEP\���)�q���u��x��B������N��;�,���+�<��(��(��(��G�r�ZU����O�k����������}�T��+���B�(��(��(������
�����A��y���A�����"�p��G�������mQ^���EPEPL��C���>�7���t�*kq=�5_�>����G������(�aEPEPQ������*9�����5�O�^���C�~f5Q_
~�QEQEQEoI�����|���
�V���
X���������|�=z�(�x��(��(��(��(��(=(��@8<#o��o�a������E���D�&�&]�<���`��)O��?����a�hz����:�����r��1Ke�D��1,�_@r=S[�O����C�\����=2�(�5
���)��#1@�����v��i�QEQEQE���^m�G�E���{���$��k������/�/���QEQEQEQEQEQEQEQEUk4D��8|���c=*�V���}�?�}����q�@h��(��(��(��(��(��(��(��+;_������V�gk����s����2��)z3���+�>d(��(��(��*�����S���'����)z?����S����(����(��(��(��+K���2��a��f����e�������1�~��=\tP:
+�?>
(��
(��
(��
(��
(��
(��
(��
(��+@�/�Y\36�����Y��yn��[��=:q��@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@�����#]����f�_�e>q`����������EPEPEPEPEPEPEPEP\���)�q���u��x��B������N��;�,���+�<��(��(��(��G�r�ZU����O�k����������}�T��+���B�(��(��(������
�����A��y���A�����"�p��G�������mQ^���EPEPL��C���>�7���t�*kq=�5_�>����G������(�aEPEPQ������*9��?�����K���hz�����+�O��(��(��(���?�������!U*���!�����BQ���O�^��QE��Q@Q@Q@Q@��x���r����������7�n��`2��?�s@f�������G_������tD~[����~��0Aq��0q�Y:�}���s�?���)��G�9�|K��VI��Nu������K���|�|;�Q����(��}�s���y��2���'����?�-?�v��x�V����@�+X|�m�lL����k���(��(��(�=?�o�?�/x���K����'��^m�G�E���{���=*�(��(��(��(��(��(��(��(��Y�7����L�����j�V�����E]�2����@h��(��(��(��(��(��(��(��+;_������V�gk����s����2��)z3���+�>d(��(��(��*����OP]����J������#���O�K�2h���s����(��(��(�/���a�]��5�Z^�������j��k����?��q�Q@�(�t��(��(��(��(��(��(��(��(���!��
�]vnl��8�5Z	7^���6��r���QEQEQEQEQEQEQEQEQEQEQEQEQEV�d[ReM��~\���*�W����,_�Q�xU�(��(��(��(��(��(��(��(��+��o��!��?�1�����R����������TQEz��QEQEQE����O�kJ�u����y�����C���1�����QE|���QEQEQEWy���A�����"�:�>��6�����WN����o�Y|�3���+�>0(��(��)���O���&�P����Mn'����G���~��R��|�QE(��(��*9�����5%G?�{������K���hz�����+�O��(��(��(���?�������!U*���!�����BQ���O�^��QE��Q@Q@Q@Q@Q@(�y��~)Ia�k����<K���k�j�8����y���W��q��S��A����
)�;Io$f7e���8�O��wZ����w�j���-"2���;ROz����^>�?��o��~%������+�-���"YU]�rv��4���6���
������9"�_������Xc���=��S�����C��o���W���z����Ny>��:?x�[>3��o��m&����WwW���H�����>�����?�?��:��>���W���~���X�6��*����Vo������7����W?������O��
^>%:��GR�\����jz^���[�Vp^k����nX�m�^�>9=�=�:��>*������:l�k�e�i����Y��L����T��j�G�����y9��8���]3����	�o(���P����y����E�c%��Z������d�8 ��V=��� ��u'�I�p�6c��@�Eg��X60�l-�����-K��;�����5:}��}�
�l�l~4f�����������q�47�|�� �9�����Y��I���� ���qn���No�����9I���@�U��~q�!���	��H�m������[��Y����-�A�dm�lc�h?m�W3<��1@h�������}����}�_��h��_�[>�b��}���|�����T�����6���[9���EVO��o�[������h�M�Oy������h�Aj����1+����)���)6<�w��=�PB��o��s#���g��hQP7���6�|������T'����C�s����'��G��?� ���in���/�|��-����-�w�Y����#3<��������}����}��]���/� ������K����D>N}N�*����<��pv�-���"}��0A�&����Y����'"7w���
��)6<�wd�=�@h��}����w��[���9�����C��g$��j����k�����0N�jH������.��}��gk����s����m�[p��������P�>�����"���i8�GOz��2��9z3���wv�7lW�s��GQM���
��E��:�o��qG������E7��l�wv�Q���z���~l��Cu��I�c�w��z?���/���_�2��8��8�|W�X����J)x�|P1�sE���J)F;���w�+7�QJq�3�8�����%ixw�F[��?���3�����������4m��8=j����c^_�������?�v�� ��6��}�����<l�Y�;���>���~����C������������t�v���@h����)�<�6��=�G�o�0 �ws�m����*����O,A�>����+��z�b
���{��,QP������S���>��0A��m-�j�EVO��0[�����3�@�o�r �3�-�f��~�������%������������-���,�U��~p�!���I��J�k�A�!�rq�wc�P�Uh���y��v�-�����o���g0[��B������cn�=����k�	'f�K3.�Y���q�S?�|�� ���%��P�*��l��X�n������?k�G'>�v?�OEWO�y��y|�il�R'�v��-�c��[��4Ua��$�A�g�[����]�3'vKc�f��'�p�X����qn�����8l�\g$���T�h��>N{���?�e��0v�-��h�Y~��>������h�M�zA���m���EV��i�A�~Kc�������cv��{��,QP�����������O�y���|�il�P�*�}�k���v>M���z�|���������Y����+�A�d��lc�(��O,A�|��u��4Uw�_�6|�3����k�G�!�s����T=^?�no4A��Ku��D�o���������Y����= �wz���
��-67��%��P�U�&������L��F�o���P����r3�j����(���rwc�P�Ut�g�w�|�q����O��0A�|�Ku��4Ua��%�<���-�P~���f~l��(�Y���<�o�>��}�_�~p��>_�[>�b��}���|��'v?�$l��h�;v���4b���m��x�0[���OH<�����@h����)6<�w��=�D�m��Bm���f�����(���rwc�R'����C�s����,QU���_�go���_z�|�� �26��1�@k��o��!��?�1�����y�9����|W��h[��7�v��;�t���q�U���S~l��Q�g�+�������7w�6qE��:�o������l�p�E4���>m��E��:�u����h|�����?�&���^ne����xu[0�����R�������|���y����c�h�9�o!(��w�)�l��sy	E/�G��v!+��� ����?�\'���<
���������>a9����0����3Y_/��v�T
���
�'#9'v;�R'����C�s����c��Uh����D��m-�����o���fF��;��,�U��|��o�n�������n����3�@i���O�����b�b
��[>��~���!�p{����Mn'����G��������/��^����E7��l��c�h��GQM;�b��=�E��:�o����n��.��9�����4���Q����q�������o�_���2(���1�k�,~���%q��ix��Vo!(���h�=�N���%�g�(��,�BU�'�CV?����*�������f��~���)�j��/rZ�EV���?� ��f��}��_��O�A�q�����O��4Uo��$q���[n?���o����|�-���,�U�����m����s���h���S���'����<��0q��}�������m-����(�=h�����JZ(������|T�]����o�],�����Wk#�S�:r+�?|�I���?��]�w�k}�_����G�ybYm�P�OB�A�k6����Z�����w���$Cx�e�����P��*�K)"�\�pFi��Q@�������q�j^����s$���]�Ry�'����?������!�B�8$����%\*7d��{-��(���t/����G���Qi���0��U���x��>��������|W��)4K&]?M3��<�����1���O��<��>�!����o��y�X���v�#�T��d�>���4o|C������C�t9M�v�\�����$��(�=x�Z(O��������"������zI����|��{�_�2^�5�J��(��(��(��(��(��(��(��(��q���:����^�=>�f�Y�k�yn[39l�`��@h��(��(��(��(��(��(��(��+;_������V�gk����s����2��)z3���+�>d(��(��(��*�����S���'����)z?����S����(����(��(��(��+K���2��a��f����e�������1�~��=\tP:
+�?>
(��
(��
(��
(��
(��
(��
(��
(��+A[����v��u\�f�@�/�J�.vn\t���@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@��Z.��)�t����k��hD�Qw/ g��*�QEQEQEQEQEQEQEQE�x��B������_\���)�q���t��qc���*(��S�
(��
(��
(��
���'���Y�����v��������������EJ(��L�D(��(��(��+��� ����?�\w���y�]���+��Tyy����_���E��QEQE���?���S����O�����U���KH�t})k�>P(��QEQE����RTs�����k:��
�������j(����(��(��(��*���!�����BR��?�������!U���%�z�Q^���QEQEQEQEPy���C��<;�i6i���j�yI-���r�d9�G����@c�M7���o��ku�����]	�E	�`8*s��T_|}��x^OxJ�5�cVx��-T������[>(�����^h���u�Q-e1�.�&�����G�ei�|+�:?��X�|�f�2���[�����'m���O����J'�9B��P�X`���O��(��(��(�=?�o�?�/x���K����'��^m�G�E���{���=*�(��(��(��(��(��(��(��(��Y����?|��z��Vj�����P&&ppz���4QEQEQEQEQEQEQEQE����
����+F����]��E]?�z�V���QE��2QEQEQE��z���������X����u`�����~fMQ_~�QEQEQE����l?���F�kK���2��a��]?�z��?�?G��:
(���Q@Q@Q@Q@Q@Q@Q@Q@�1���``�&�O�1Vj��}r��������EPEPEPEPEPEPEPEPEPEPEPEPEPk��>pb���<�p��Vj���v����6����4QEQEQEQEQEQEQEQEW!���
C�\�c]}r-���?���f5���!���0����O(��(��(��+7Q�\����f�?�����s?�g�>��?�cG�(���3���(��(��(������o?����Epu�|?��m��w���/�Q������~g[EW�|`QEQES&�P����O�M����?���Oc�W���-"�����t�@��(QEQETs���MIQ�����5�O�^���C�~f5Q_
~�QEQEQEoI�����|���
�V���
X���������|�=z�(�x��(��(��(��(��*���g���F�HA�O(�rz�Y��w��W�t����Q^�W1J���|#��R��Q��i��X�>M�i��t[�9���S|A�������|=������m#�t����`<��{P�w���S]O���"^��[�z����%s�������]*�;��b�#�$�����dh�l�I,`y���R�c���52)hRH�2:�V���@Q@Q@Q@z~5���^������k�OO��������"������zUQ@Q@Q@Q@Q@Q@Q@Q@W�w;z����V*_;��~����w�OEPEPEPEPEPEPEPEPY����������k;_������U�����o�K��%Q^��!EPEPEPP]����J�����Y>���K��GV��������(����(��(��(��+K���2��a��f����e�������1�~��=\tP:
+�?>
(��
(��
(��
(��
(��
(��
(��
(��+����
����n3��z�PC�}�������y�������(��(��(��(��(��(��(��(��(��(��(��(��(�
������C6�+��b���~�~���������5=QEQEQEQEQEQEQEQE�x��B������_\���)�q���t��qc���*(��S�
(��
(��
(��
���'���Y�����v��������������EJ(��L�D(��(��(��+��� ����?�\w���y�]���+��Tyy����_���E��QEQE���?���S����O�����U���KH�t})k�>P(��QEQE����������=���Y��%�m��4=W�cQE����EPEPEPV���
X������oI�����|���
��H���/C��������(��(��(��(���M��<��:k��\�o���U�E���M�o�l���	�����m;�p|"��_�[i�jV��Ki$P�E���t�0,##�j�|3�i���<]�,�v������x�^T��L��T��+�[i5(4o�E'�/;�(��B���v�o�YB�{�S�m�qSQEQEQEQE!����|��{�_�2^�5�I=?�o�?�/x���K����TQEQEQEQEQEQEQEQE^�<��t�Fp3��Uk4T��8|���t9�@h��(��(��(��(��(��(��(��+;_������V�gk����s����2��)z3���+�>d(��(��(��*�����S����J������#���O�K�2h���s����(��(��(�/���a�]��5�Z^�������j��k����?��q�Q@�(�t��(��(��(��(����j�v������#+\���>�����,/,M��������2�|u���A�[����m��w0GtB�O_�km��W����<Yk���Z]]<-q�����=�+�����8t(����7VYN��Z���l��e�a�;I����A���	o�j�6s8�Gqr��}9�_��Y���V���)yX�@����zH/�^���%��a�r��z�[E(��r�`��8�z���� �2�Y��!�t��m�����1�(�s��sY����+�t��n���Q%�v���v �1��_��&�uu{��^^j-�Gl�+K})]��������X2�� ��;����{���V���$���,�6��$��Pv��@��������6q��j�U�E[��W��r���VhZ������M.(��5��F9�R?SX�7�u�sH�b��������e$�W6��	�H�co�#5o����c�t�F^	TJ�%��F�uPFG�Z�4-z?�K\��O�W�t[C,�������-�P���-j��P�{���I%����\�U�]�:�pj�����@�����m�,�J�����
��C����6�<f�x�J�5���Q/�K���"�k
���Y@�3������|]����i����>����5���)P���
����@�it��0��I�Y0�2?�qW,��uutU����IR.���LB-��H<����]r���J�36���D�
��c�<t����R�@�\�!���\K�5�.���G�l��%7g�y�zw�5��
����{cyM���L��=���X>��N��	q�}�y�A��8����g�5�@Q@Q@Q@Q@Q@Q@Q@Q@Q@����*��S���!V*��+�w�����f�
(��
(��
(��
(��
(��
(��
(��
(��
�<[�!H����k��C����������p��8���fQ^���Q@Q@Q@f�?�������G�r��^ng���G�����c��"�Q_&~�QEQEQE�|?��m��w�������
�����A���*<����_/��h�����
(��
(��
d����������?���S[��y������_�>����(QE
(��
(��
����MIQ�������|�6����1����S����(��(��(�zO�����?�UJ����j���c���T~$gS�����QE{���EPEPEPEPA�E�yN����W><����=1���"�����Y;HG8$*s���8�����O�����mkT�����!��O��R�ko ����q�4�W�|%��h�H���n�/���4q�����A��h� YY�<�@v8���P�c�p�S0������5QEQEQE!����|��{�_�2^�5�I=?�o�?�/x���K����TQEQEQEQEQEQEQEQEZ����Qc����w�?�Y��l��yi�8<�'<��EPEPEPEPEPEPEPEPY����������k;_������U�����o�K��%Q^��!EPEPEPP]����J�����Y>���K��GV������d�E���EPEPEPZ^�������k6��;�#-��v�������#�3��������Q^���QEQEQEQ\��uK�-Ib���� �
9>������O��o�����������SNqi�t������/�ks�o!�p�*�:���c�S�|(�R�W_��:���e�fD,�A
rN;��o��S�~���_���T������*��>���
]���,�^��m~�S��7_m�e����Q1U�9+�:zS�<q�������I�T��woQ��B0���(H���o�������������������G�'���.�����xU4�'F�4[��].e����r�OP���Z��<�]x�_���ky�bx��)��07��q����J��������o���������T�t��?��v:lV�|P����}��c���W��V�����?������
?��O��o�����O��
]���:+�oK��)�����@�A���q�
g�&:������\�u5�������	 �T+��^t��i��X��'
��yy�����h��������h������y���*vGg�.������	���?��~��)?�0���������y�}r�d��~��_�zo�&:������G�&������^eE\���.������	���?�����Q�	���?�����W�QG�*vA�������G�����V�sg'�m#���V+�_�,A�������i��	��1�*���6��(�1
(��
(��
(��
(��
(��
(��
(��+_�f����n_�9������k�E�&T�������*�QEQEQEQEQEQEQEQE�x��B������_\���)�q���t��qc���*(��S�
(��
(��
(��
���'���Y�����v��������������EJ(��L�D(��(��(��+��� ����?�\w���y�]���+��Tyy����_���E��QEQE���?���S����O�����U���KH�t})k�>P(��QEQE����������C��k:��
�������j(����(��(��(��*���!�����BR��?�������!U���%�z�Q^���QEQEQEQEPzQ\�����)�8���v���v���g8U����0����u���'���w�~����f�����cj>
���[q���X���YM�ZlLe�A�����+����E���!�i�2|�c���p��rs���Lk�|(�lf��|M�{�����h��f�p���������XaH�DP�=��HG��x��z�����%������"yeg���1��tO��E��e�������c�����P��z�^k��5��{���/�[���b�>���]���S�^[�[j:��2O�wU����P�!-� |��XX���z�Q��k�x�xC���^^���ZG�>Q��{��-�#��y�<|�2�M�j6*R9���e=�������'�Zo��U\�����K�[�������[�����p��(k�>4�l<C��O�������/�tO�k
�����8��e��xO�Z^���:��Zc��aDa�,�O�>�\V��~�������g��.q�������*���������\���OO��������"������zI�����2_��+'������RI����a�1�~���V~�5�c���7^P�M�p�g}������O��o���fc��4Uo��'�n�F�������&��w�6=�@h��}�+�1��pn���No����0�9A���@�U��~q�a�����H�m��������Y����-���dm�lc�h?m�W3<���Y��?�v'�m�c�����y��0�|gpl���*���ra�s�wc�R'�77�`���hl��h�Y~����o�&c���7�<���������5Z�O3��6�����=~�7�|��`�9�����Cj-�>H��|���oj�E@�k�@�a�r3�wb��_�w�|�q�w{P�Uh����L���
�����o�����������4Uc��$`��g�����<�o�>����Y����<���6`gpl���������C���'����7��`���hl��i��[��
��0��4Uo��$�������������&�������(�ZO���Q�����u���7���6|������@�U��~y�a�����I�~4������Y���@W_�QV��Sn0y�p�������[��m���WO�^�U��/FqtSN��������|�/��)�6{b���(��:�o��J>lv�/��)�6;f���(�r���.���O�K�g�*��f�8�+��R����_�4���4eQK�l��{��*��|�BQK�{���s�Vo!(���N1�4X9���S�q����o!+K���2��a��gq���
w�$6^V7��n����W��S������XY�����w�����9�����+#9w�{����P/���w|��`���������.��}��Y~��>������h�M�Fn�xm���EV��i��7���lg�����|��
��g=�@����0�����Z��O���7m����zd�0��q^���|�%~�C�����G����na��:�o�������.�c������>m��E�a�SF�����l�q������|�j�Z����������8�����'�~��J�
J���c�i8�|��S��(�8��c4X9�������g�)�9����c�h�9�+7��>
��b�����o�7���`A���w�s��x�kr?�e��0v�
��k���8���y���2�Y~��>������h�M�z����m����,�U���Zl0o������,�l��Y����b�����|��v?�"}��>a���Cg��,QU���_�6���m
���>���LfF��(�X���]�3'vCc�C��jyf�����Y�������������zQ���|�'=����@�U��f[�0cn��{f�~��>��6`6=�@h����OX<�����C}��M�
���l~_��Z����(��X
�ToZ�"��'����s��{T��h�L>N{����'�������������"}�k���8�v���@h����Kn0y�p����\<���
�P�*���jyf�v>}���R������|�3�6}���_�90�9��*H�����0v�
��h�Y~�����6`6??�|��y��n?��,�U���Rl0y���l{b�>�����7n
��(��_�>S�������O���y���w�X��'�v��`���hn�����-���dm�lc��,�!���
C�\�c]1�o��0y�9����|W��h[��7�v��;�t���q�U���S~l��Q�g�+�������7w�6qE���:�o������l�p��E4���>m��E��:�u����h|�����?�&���^ne����xwL�>��*QK�{��3���?M���Q���c�4X9����1�4��J���%�v�g�)�9��������o?����Ep�g�+��7�����b��������J��/����e|,�����}��i����A����/������s����c���������3��������%�<���
�w�Y����#����
�P�m��Y�v>}���j�L��C���D�l��0l����������������?*kq=�<_�>�����g)~oj�.|�/��)�6;f���E���:�i��|���.._1�S~l��w|Qp��Q������|����l�����5�G�?C|:��=Q�(���1�k��~���%q��ix�����%�q���{�����J)x�|Q�|�`����?�������!Ux�|��'�6=s����!N+TEY{����*�m����;6����B���}�3��
�|��~|Y����#�<����q�4?�|��m����l~f���l��Y�n�������h���C���'����<��6`�hl�R'�v?�m�c����4P3�z�@y���]>��w���^ �Y������O��^�T5mJ���kN���7��sp��d����mx���?����;���_k5�
7�WW�@��|�2�����������W��
?������|/��z����P�����V<�
�|��H�����UR����;������#o��gi�Xo�G!����A�XV^�o������v�%���������#nv`t����Q@m����G���7�U�:����R�e�DgQ�]X��`}*=��&�_�K/x�N�G��bd����L�F�#�������P�x��z���K���#��T����,g��[��n�2q�oJ�M��>������G�A��9M�6_iY����1^����N�C���6�#�"����d��k^�z~5���^������h����(��(��(��(��(��(��(��*U�|�8��X�N~\�S�k8��;�W�3��zd����(��
(��
(��
(��
(��
(��
(��
(��
����u����Y�������������L�
^��(�����
(��
(��
(��
���=d�T���z���k�
^��:�_�T���4d�E���EPEPEPZ^�������k6��;�#-��v�������#�3��������Q^���QEQEQE����C�\W�������a���3X����>o�iz�QZ�Q@Q@Q@���|�j�Z�?���Uk�q��3�?\���_K�(���=`��(��(��(���"������������F�����8���y���0��+S�(��(��(��(��(��(��(����e{b $>��q���������Z����)���
�@Q@Q@Q@Q@Q@Q@Q@Q@r-���?���f5��!���
C�\�c]8_�X���
�(�T�B�(��(��(��u����iVn���?��73�v#�xs�F1��R�(��?Q
(��
(��
(��
�>��6�����W]������q������^m��/��u�QEz��Q@Q@2o����T�d���������<�~��R�/�JZ�O�
(���Q@Q@G?�{��������w�t�u>	za��U���QE|)�`QEQEQEU�'�CV?����*�[��5c�_1��B�?3��K���(�������(��(��(��(��(��(��(��(��(��(O��������"������zI����|��{�_�2^�5�J��(��(��(��(��(��(��(��(��k��[���[#9�U��fc���~��n=���@h��(��(��(��(��(��(��(��+;_������V�gk����s����2��)z3���+�>d(��(��(��*�����S���'����)z?����S����(����(��(��(��+K���2��a��f����e�������1�~��=\tP:
+�?>
(��
(��
(��8���a���3X�����/�q_�k��?�G���/P��+S��(��(��(3P�����U�Z��|�j�|n;��~���7���zQ\g�QEQEQEzW���X���?�#[����� �~O���{t�C����5=_�QEjr�Q@Q@Q@Q@Q@Q@Q@����\��^@��U��~c��Sr�����j�S\���sp��O�q:���Q��h�_��_�cMsGis#Ov
�P��F@��wW7�?I��+�%��w���m���Rw;��U�Rz��R�u������1�����E��� m���tW����w;��uX�����n.&�2����O�������x�xz�T�&��
���
,��Q�� P�����[���������U��c3:����Nv��885�PEPEPEPEP\���)�q���u��x��B������N��;�,���+�<��(��(��(��G�r�ZU����O�k����������}�T��+���B�(��(��(������
�����A��y���A�����"�p��G�������mQ^���EPEPL��C���>�7���t�*kq=�5_�>����G������(�aEPEPQ������*9��?�����K���hz�����E|)�`QEQEQEU�'�CV?����*�[��5c�_1��B�?3��K���(�������(��(��(��(��(��(��(��(��(��(O��������"������zI����|��{�_�2^�5�J��(��(��(��(��(��(��(��(��n��l�&&ppz���5^�����0��Z�EPEPEPEPEPEPEPEPY����������k;_������U�����o�K��%Q^��!EPEPEPP]����J�����Y>���K��GV������d�E���EPEPEPZ^�������k6��;�#-��v�������#�3��������Q^���QEQEQE����C�\W�������a���3X����>o�iz�QZ�Q@Q@Q@���|�j�Z�?���Uk�q��3�?\���_K�(���=`��(��(��(���"������������F�����8���y���0��+S�(��(��(��(��(��(��(���~����1 ��)?�
�U5;�i��;m�R��@5G������o��r�wf��R���~���E�6���PI'����-�� ��Z���4[��:|�C�����Q������A���D������(���D������*}�;�����G�3��.���.�I�D�=�OFv����!%�$���q�[~����m�\kF�4��
�lD`�����q��j�[����
��[�����
�{HwA�\G�?���
���E������^�n��a_������������#����F9
#[N���������[����
��[�����
�{HwA�\G�?�����g�ox\xzG��L����a*�<�EF$1/�z���t�o�Kt?��O������KtO��O�����i�>���G�3f�����=��pCz�$�krO�Z�JJ[39��7i��B�(�3
(��
(��
�<[�!H����k��C����������p��8���fQ^���Q@Q@Q@f�?���ZU����O�k����������}�T��+���B�(��(��(������
�����A��y���A�����"�p��G�������mQ^���EPEPL��C���>�7���t�*kq=�5_�>����G������(�aEPEPQ������*9�����5�O�^���C�~f5Q_
~�QEQEQEoI�����|���
�V���
X���������|�=z�(�x��(��(��(��(��(��(��(��(��(��(��C���6�#�"����d��k^�z~5���^������h����(��(��(��(��(��(��(��*_7��v����w�OU������J�`�4b�(��(��(��(��(��(��(��(�����]��Eh�v�� +��?�����S*����8J(����B�(��(��(�����Y=1S���'����z?����S����EW���Q@Q@Q@ixw�F[��?�����������#WO�^�8�����G����AE{���EPEPEP���/�q_�k��W�!���+��b��G�h��O���QEj`QEQEQEfj��?���kP�����U�������s&��}/@��+����(��(��(�J�W���'��k~�<�"�������n����|3�������(�NP��(��(��(��(��(��(��(+�������<�~��J���O|5}�����y]y���Q�~�EW	��Q@Q@Q@�?�t���O�^�:W���0�������ht�K��������(���<��(��(�C������������<[�!H����k��C��aQE��QEQEQEVn���?��*���'�������}
���>��*QE�g�!EPEPEP]������q�������� ����?�]8_������e�����(�\����(��(�M����?��L��C���5������JZE���K^���EP0��(��(����w�t�����������/Cl?����3�(��?l
(��
(��
(��
����j���c���T�zO�����?�UG�Fu>	z�EW�~xQEQEQEQEQEQEQEQEQEQE!����|��{�_�2^�5�I=?�o�?�/x���K����TQEQEQEQEQEQEQEQEZ�<��398zU��f#w�X���ny�(�Q@Q@Q@Q@Q@Q@Q@Q@gk����s������@W_�QWO�^�U��/Fp�QE{g��Q@Q@Q@Aw��}*z���=d�V5��/G�X/���_��EW���Q@Q@Q@ixw�F[��?�����������#WO�^�8�����G����AE{���EPEPEP���/�q_�k��W�!���+��b��G�h��O���QEj`QEQEQEfj��?���kP�����U�������s&��}/@��+����(��(��(�J�W���'��k~�<�"�������n����|3�������(�NP��(��(��(�)x�U�3��^q�{;�	.�q���W���o��;[��5/?�<a@\����iY�h�{�%W8�{��8��|[�6���b;�Rs���S�Gg���q�OV4x����s����j}��!��'0�����f�dT�����6����]���i5���n��LT6SA3N����������}N�XH��Ks�43F�Cd�
��G\���m���{v1�jr4�S;RI�� �g'��a�R�f����MWT����R��w$�%���p��Wh������>$�����-�8�/K"�bi.�;��q�RxA������Q*�������gt�6���3��U-sA�|c��CM������S5�#���o�#�h;���t~#���S�8������+���"��h�@�^�\?�_��G�}�]��t�mW�"+�=�%h�0�����w�2�L���N?u��=+���<M��F��g�W���u��>$}FI�9��Q\'�QEQEQE_���a����?�z��^K��������:����,������@��+��B�(��(����R��������o��!��?�1��/�,w�Y�EW�x!EPEPEPY�����v��7Q�\�������?��<9�#�?��EW����Q@Q@Q@w���y�]���+�������o?����Et���/6�u���:�(��s��(��(��7���t�*}2o����T��{j�t})i���-{��Q@��(��(����=���RTs�����k:��
�������a��(��?l
(��
(��
(��
����j���c���T�zO�����?�UG�Fu>	z�EW�~xQEQEQEQEQEQEQEp� ��>��	��9s��LA|����y�$|�.p<Vw�<_�]+�~���\�^[5���b�`�������^���T�,�].�Q�.#���B��!�P?�N��~��<{���������oZ�v�;�I�3������jx����6��_���u&��]������,�|���?4O����<����^���N&�N���1+����OC��n����rh�y��E��L�sh��R�\����s��������l���z���4�	�>yu�UZ_:�&+XW��S����Y��)�w�<_��=��R������&����c��������/�W�������|{���1��������n�����^m�G�E���{���$��k�>��y,|@|+y�Ae�	���-��M��H(�c�h�(��1uu��j���t!k[F��N�$���S����{A���F���Y��b����y���m�������M��w��ls@h��������r����9���>V��������q@�U�]��{C������"�?����j�_~h�X�%�4fF������/|��A�g������,�U�^�O-�
��r63��+��4ymN2N}���*.���C�����\������;v��{g��EVA{����~>L#c���^y'�������q���5Z��o;�B���d�'<�^yI�����Q���j�����yB�6���#|��hF����������*wc����w�>W8N�n�=Z1{������.�n����E����3#i���4f��E��0�y���lc����������#c>���*���0ymL�S�~����G-���N�}s������{y�\�T�=��"�-����a[���,�U�y���f�������Rmh<�w��ls@h�����h��nS�����_h'# ������'����������)����1y���������4f����]��EXQ{�������B�=��P���#�cE�6�8�GNz������o����]���?*�������|�c�����b��<�������)�79��|����/��)�6;g�A���_1���z���~l�������i3�b���u/G�X%��O_��4eQG�K�{��*��|�BQK�{��;��y�7��Q�|��;�����(���ix��`����;�#-��v��w����3�	
����h�����P^��1�/��N����QU�^mO)�/��[��4�~x���YN�~��OE@���sC�d�;���c�?��>]�z��@h��/|���y�m!���^y#
��������@h��/|���o���lg��W���h�n��9���9_��a���3X����3�hn+�)q��&��������/�h��J���E4n�8�������na��:�o����Pwq�~T\9|�QM;��1�����J.+y���7w��G����/������v����������q�5���|D�O�2y[I[��P1�4q���;�7�QG��^;f����J)x�|Q�{��y�7��R��4�c���7��^
��b�����o�5��q�����L��w�=�u����[�h���O^����_�����������,QU�^�O��������h������wu��q���R�Y����
�;�����Y����07nS��9�T]}�����B�v>��"�4�������zu����Kp�YcR����.���i�����Z��-��<���G?�����~����	��#����NIE�w�%����?�;�4�_m�>�����\�����Pwv��]W�����o�G]�	}���O��������������"wg�c�G���(��>�������_m�>�����G�%����?�;�5�����Q�c��G����V�du������������	}���O����r?6;g�G����W��>�[���k>%�������P�F@$������s�w����H����3�T��c�8j��P�����Ec�����k���<�BQK�{��3����BQK�|�q�s�,�AEc�iN;f�������������:�����F������y���3�� /<����������zX?��1�;����Y��0���cA����c�� ���MF��z��5�xE�*.����>Nz;���)]��{C������X�� ���c@N��j�_~h��m�����lc���\���)�q���t�^�+���������\������aB�@��q���p��8���Lj)�vx�>�|��b�K�
��QM����n��?*.>_1�S~lv����������:�i���6���Qqr����G�r�Z6y�>����7zv�72�v~���0�����R�������|����y���������h�syq���8��Vo!(����3���BWy���A�����"�N3��x\�>���_<g�RO����_�G����e�o��(�]}�mh|����v?:]��{C�s����^����Uh���h	��mS����(��[sA�dm![��h�X��$a��7rv61����bym�|�F�}��4���?���TN/<���03�N}��e[�-�������)���<�~��R�v��t����?*�.|�/��)�6;g�G����Qq�����wv��G����Qqr�����g�>�
���Qp��Q������|����G6~���CY�~��7��}z�"��|v�������?7�QG��8��Vo ��������;7��R����������%[��5c�_1��B�q����V?�lq���?�S��R^��=v��������v�S����(��_sA�q��l{���O��4Ul^�#������������Zlh7c��6?h�]�������)�{�����>�>Nzm;���(z*��1����6���zD��h7c������@h�g���
(��
(��
(��<o�F������3���
�2��_������sd�x�?�u���k�^��U�iZq��r��2T)�NO Z���9�g�|C���xk����&��\K���*��:�z�I����Q���^8�������t�2�;y��Ga������@��kSx�I���>�������/�;��n�FN	��������u�
R���k��:$�i��;�q%����W�����R����k������/�/������y����E�c%��Z��(��
(��
(��
(��
(��
(��
(��
(��
�j�����bV�>\�V*U�|�8��X�N~^�=a���Z��
r���"�U<��8'
{��r�@��i�69���PG5�������
�G����������t�h�����.%�M�A��X�.q�fP�t5��I{��};Ay���H�ca���u]�dV�����~7�4McS�V������[,2[��xS�����MjI��
C�5���	�a���$��l2�`l��z�D9��m��|"��1y�3�Q\�������gg�;���n3�9�T�L���i��.
�^n��������i���\�[X��D��1f=�PO���k�<m�+gG���%���g�[Ic0X!Y�{�+�4�s�����:v����{fd0����F��9V��P�(��
(��
(��
(��
����u����Y�������������L�
^��(�����
(��
(��
(��
���=d�T���z���k�^��:�_�T���4d�E���EPEPEPZ^�������k6��;�#-��v�������#�3��������Q^���QEQEQE����C�\W�������a���3X����>o�iz�QZ�Q@Q@Q@���|�j�Z�?���Uk�q��3�?\���_K�(���=`��(��(��(���"������������F�����8���y���0��+S�(��(����������k����	w�\�p5�a>x����QE�y�EPEPEP7_��'���+^��=��v�+���G���?��z��EW�}�QEQEQEC���O����u�C�y.��#��_	����J��>[;�,=�(���
(��
(��
�<[�!H����k��C����������p��8���fQ^���Q@Q@Q@f�?�������G�r�^ng���G�����c��"�Q_&~�QEQEQE�|?��m��w�������
�����A���*<����_/��h�����
(��
(��
d����������?���S[��y������_�>����(QE
(��
(��
����MIQ�������|�6����1��Q_
~�QEQEQEoI�����|���
�V���
X���������|�=z�(�x��(��(��(��(��(��(��(��(��(��(��C���6�#�"����d��k^�z~5���^������h����(��(��(��(�����<'ng�G���H��e��|��8����u����)��$��VP����`xo��(�u+=:��1<��X�9�\������::+������,�M���qm���$r�����#���(�J���Eq�
T��4�;��TM�����3����I�����u����[{��X�nR2
Y��q�~w���fo��=>�f���7�V�"����t������d������T�vva0�S�;is�k����]�����6�[��%|�����SP��������G����?�_�����������?�_q������-�����������0�*�y�G�v���GQY//o|�%./��w�<�������u�?���?����������������o��������u����.u�_\j�y�x������� �d���zV���ky��j�����dX��B �/C��J��MC���������������>����?�_q����i�}/P�
����[�n0��h��jj��u���h���?������������I�:��_����SP��������^��Id��6�M#��~Y��~s����
�����������}llQE�yaEPEPY����������k;_������U�����o�K��%Q^��!EPEPEPP]����J�����Y>���K��GV������d�E���EPEPEPZ^�������k6��;�#-��v�������#�3��������Q^���QEQEQE����C�\W�������a���3X����>o�iz�QZ�Q@Q@Q@���|�j�Z�?���Uk�q��3�?\���_K�(���=`��(��(��(���"������������F�����8���y���0��+S�(��(����������k����	w�\�p5�a>x����QE�y�EPEPEP7?��'���:V���{���d��y��#�~�����=@��+�>�(��(��(�����������:�����C���O����u�C�zX?��-���EWq��Q@Q@r-���?���f5��!���
C�\�c]8_�X���
�(�T�B�(��(��(��u����iVn���?��73�v#�xs�F1��R�(��?Q
(��
(��
(��
�>��6�����W]������q������^m��/��u�QEz��Q@Q@2o����T�d���������<�~��R�/�JZ�O�
(���Q@Q@G?�{��������w�t�u>	za��U���QE|)�`QEQEQEU�'�CV?����*�[��5c�_1��B�?3��K���(�������(��(��(��(��(��(��(��(��(��(O��������"������zI����|��{�_�2^�5�J��(��(��(��cK3�>����#�E�|3���	�����}�/�������g����>��X��*V3p��A�~���tc���k�
�Yh�7�]�E|�r���r�K0o����X�������[�^i����e��j�m�
�m��+��4�@�k]���s#G
��8���M���q}v�ur��L�
 _�	���,����i>������~���9��o�
���1�G�!���z:�Em"��8/.�����9 �N3�e[�G�v���[h�dW*��U�V�������n5k������]`��Z0���~�Y�������c�+�}~o����a�DJ4�����N��c�5�>	����k�d�k(�[ ��6��g�������}�Q�t���34�#1�L���l,�"�W�(0��.�k��M��I�v�g\���|_��?��^]�VW�%� o�s�W3�3��Z����Q�EW�}pQEQEQEW�xC�E[?���k�k��!�"����C5���#�<L���z�����+�>L(��(��+;_������V�gk����s����2��)z3���+�>d(��(��(��*�����S���'����)z?����S����(����(��(��(��+K���2��a��f����e�������1�~��=\tP:
+�?>
(��
(��
(��8���a���3X�����/�q_�k��?�G���/P��+S��(��(��(3P�����U�Z��|�j�|n;��~���7���zQ\g�QEQEQEzW���X���?�#[����� �~O���{t�C����5=_�QEjr�Q@Q@u��]��#\
w����.�����,'��0�"�
(��8(��(��(�����=��v�k�������@�����C��w���Q^)�!EPEPEP��F?����������:�|'��Z+���,�l����
(���(��(��+��o��!��?�1�����R����������TQEz��QEQEQE����_�kJ�u����y�����C���1�����QE|���QEQEQEWy���A�����"�:�>��6�����WN����o�Y|�3���+�>0(��(��)���O���&�P����Mn'����G���~��R��|�QE(��(��*9�����5%G?�{������K���hz�����+�O��(��(��(���?�������!U*���!�����BQ���O�^��QE��Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@z~5���^������k�OO��������"������zUQ@Q@Q@q>(�m�?�/o�v��x��Cm�\���V��?�_3h��G���h�P���=�F��?*Z(6�A�Q�z���M��~Tm�����h��d������d�������z�����|��_��h�����p��(��(��(�N������?�����xC�E[?���k��G�x�������EW�|�QEQEVv�� +��?������u���t�5�e[�R�g	EW�|�QEQEQET��'���.���O�c_�R�����z��%��&�(��?h
(��
(��
(��
��������#Y�����l?���F����Lq�����W���O���(��(��(�7��_�����-mx��C�\W����f����x��K�(����(��(��(����?���Uj������_��y����M�"�^�EW�Q@Q@Q@���� �~O���`x+�E�?���B5�^�����,g��OW��QZ��EPEPo�@����]��� K���k��K	�3��?���(���
(��
(��
(��!���i?���Z�_��'��k W�������
�����(����(��(��(���#��_	����J�]�F?�������`�|�w�XzQ]��QEQE�x��B������_\���)�q���t��qc���*(��S�
(��
(��
(��
���'���Y�����v��������������EJ(��L�D(��(��(��+��� ����?�\w���y�]���+��Tyy����_���E��QEQE���?���S����O�����U���KH�t})k�>P(��QEQE����������=���Y��%�m��4=W�cQ@����(��(��(��*���!�����BR��?�������!U���%�z�Q^���QEQEQEQEQEQEQEQEQEQE���^m�G�E���{���$��k������/�/���QEQEQEW��
��r_�]�q^'����%�����!�������+�<0��(��(��(+&��>���&��>��|����g��������4CEW����Q@Q@Q@zw�?�U�����f���;��*�����3]�?�?C���������(��C���(��(�����]��Eh�v�� +��?�����S*����8J(����B�(��(��(�����Y>�=Aw����k�
^��:�_�T���3&�(��?h
(��
(��
(��
��������#Y�����l?���F����Lq�����W���O���(��(��(�7��_�����-mx��C�\W����f����x��K�(����(��(��(����?���Uj������_��y����M�"�^�EW�Q@Q@Q@���� �~O���`x+�E�?���B5�^�����,g��OW��QZ��EPEPo�@����]��� K���k��K	�3��?���(���
(��
(��
(��!���i?���Z�?��'���+���G���?��z��EW�}�QEQEQEC���O����u�C�y.��#��_	����J��>[;�,=�(���
(��
(��
�<[�!H����k��C����������p��8���fQ^���Q@Q@Q@f�?�������G�r��^ng���G�����c��"�Q_&~�QEQEQE�|?��m��w�������
�����A���*<����_/��h�����
(��
(��
d����������?���S[��y������_�>����(QE
(��
(��
����MIQ�������|�6����1����S����(��(��(�zO�����?�UJ����j���c���T~$gS�����QE{���EPEPEPEPEPEPEP�������nt�2S,�Hd��� /#�Y�M�K�?
t�[[�g����H�����WG����������'�7����7��j_���7�<��^k[k�����j7Q��j���^�x��=��^&���_��{K��������>N�_��@}1���?������"�nz��������%����7�a�u����@�9�������1-����y.��tv�/W*:�����Y��&�w��e���:�����m��������:#��#���68�������c�s�]8�h��� �
����	
�������OO��������"������zOo��C����'��x_T��,�� �
����}�'r��cc��=����c����uk�Y�D@J�B��OP	8�:-����	~6}y�4Um��N<�|��|��}3C-���%�?;����c��EV�o	_*H@���2r{��s-��2�D!��J����'�+���o������[�1/$F>p�s���D&�y���Z��@�]X_�8��|��)�6;g�G����W�s���E7��G�G��1�Qqr�����g�>�
���Qp���)�6O����������n����W��Q��e�c�O������Z+��g��������� ��1���G��k��~��������=>������^3���G�����s>�J)F1�?�'z���s>�^���l����28��^��5��^-����)8��s�5��_�~�������_��[m��A�3w�8��42�yHXC�����9�L�R�ZE�;<�!/��2r}��2�}��!FT������'��U���y"1s�����c[�������20}��5����
����*�������������Fj���#��c�[1@##�=j��k�������.�i���Pwv��^�����E7���J�������:�o��q�Q�c�������u�����������/�����=d�T�6{c�P�n�4��c����u/G�X%��O_��3*�8��t�g������~���1(��=>�w���a��1(���I���4����QJq������t���������l?���F���C��hh��l��y�ia�Z�/}k�����������:��T�����9>���n��RH�YR�>�����I��n��KI�'
���4�����I	���>���*����������#8��m���7w'�8��4f���xcM��91�	�������rBpd$���r�+������f�k_�!�����yK��&��������/�h��J���E7���������na��:�o����Pwv#�����:�i����Q�n�����QM���Q�c�����|��C�>��V�_�������q�A����������d��
J������tq��k��g����c�s�����t�����R���Z8�C���g��E(�p:A�r~�[�9�c���"�������o��?�^?'|�R�w�=��[���-�I	;v�Fl�^�/����\g��OW��(����S���m"3���h�y��������8��5��Y��:�yhXC��&2A�sK"�_.H@���2r{��Tn���$^N~���\�"��Iy!1�����ho�@����]���CE��D�0�N6!�u����3��G��x������)�6;g�G���v\��|�QM���1����l}(���:�h������������E7��Q��7����r���������@�k����co�d�m�s_9�kR>�����z�u��8�|�iN;���o3���a(��=>�q��Z-����Q���t���h��s>�Ec�iN;����s>���F?�������z6N�c���z`�#9�S�y�c������8��5�`�|�t�R��f���y��Ia��c8>���E�%|�!h��2r{���<"�[��ed����(wc��D[�1/$F>p}��X��"�2XI��b20}��-��������Dg}3@k��o��!��?�1������3''�8��5���J5}�����;�t���q�U���S~l��}(��v���K�
��QM���Q�`�3�����E7��l�(���g�E���u����6{c�E�o1�����O�kC��l}+:��rn�����_���C��f��V�^3���G�q������o3����c�?���h��s>�QG��S���t�����R��:8�C��;y�3��������o?����Ep�g����O-���������!���G���2��<���-;~giE@�u��VH�YR�v>��V��%����C�ns^����Uh���%���.������[�%�K	�#i�\���*�[�$,>fy>Y�>�����l������O�4f�7���t�*����<�!	��P����*�nrd����v���i���<�~��R�v��t������{�>[��u�������Q��/��)�wb?*>l������c�������wr?*.�Q�����4��F~����w�:��G�?Cz����1��m�?�(�p:��y���>�QG��8��J����z��3���N�a��1(��=>�q���y�3���zO�����?�UN1���W4�l����#���8�Qd�%���Uh��o�$��v���~hU���4��8�Dg�#5���h��o<�<�|���,�L��xcM��11����EWu�.�\����	$��K����"�s�v����'��������8
�n����|����Dd}��4P3�z�@Q@Q@Q@W��|h���O����������|��n�
O<`�����_|+��^�|/g����j�(b������n����O�6<���K[��-���i�Rykt���~�8��=�-��&��_x�M=t�Z-;L�������w�N:c���K��8�?Y���_�<���Ztm��M�;������������2�u�Y��n��{]:���i�=�p���^�E!����|��{�_�2^�5�I=?�o�?�/x���K����TQEQEQE�x��Cm�\���k\W���6����ua�pc���2(���(��(��(��J������j������6����|+��/����E�'��EPEPEP^���l����1�N������?��f����3��u��3n�(�P�0��(��(���@W_�QZ5����
����*��k��������(�l����(��(��(�.���O�OP]����J������#���O�K�FMQ_~�QEQEQE����l?���F�kK���2��a��]?�z��?�?G��:
(���Q@Q@Q@o���0�����Z��_������5�^�����?���QE��QEQEQE�����v���C�>��V�7��?S����E���(�3�
(��
(��
(��=+�_�,A��������W���'��k~��?����X������
(��9B�(��(�
:���.������o�@����^��g��zQ]��QEQEQECs���Y�k�������@�����C�.�w���Q^)�EPEPEP��F?����������:�|'��Z+���,�l����
(���(��(��+��o��!��?�1�����R����������TQEz��QEQEQE����O�kJ�u����y�����C���1�����QE|���QEQEQEWy���A�����"�:�>��6�����WN����o�Y|�3���+�>0(��(��)���O���&�P����Mn'����G���~��R��|�QE(��(��*9�����5%G?�{������K���hz���QE|)�`QEQEQEU�'�CV?����*�[��5c�_1��B�?3��K���(�������(��(��(��(��(��(��(��(��(��(O��������"������zI����|��{�_�2^�5�J��(��(��(�+���o�����Z��O�!���K�k��C��/��EW�xaEPEPEPVM��}�kVM��}�x��������_����h��(��?G
(��
(��
(��
����g����y�zw�?�U�����f�0�~�����_��tQEz���Q@Q@gk����s������@W_�QWO�^�U��/Fp�QE{g��Q@Q@Q@Aw��}*z���=d�V5��/G�X/���_�2h���s����(��(��(�/���a�]��5�Z^�������j��k����?��q�Q@�(�t��(��(��(���|W�!���+��b�����0�����Z�h�
7��4�B�(�L�(��(��(��C�>��V�j��?�����������d��/��QEq��QEQEQE�^
��b�����o����X���?�#[����}������QE��QEQEQ���	w�\�p5�k����F����<\����(���<���(��(��(��������s���Y�o7�$}��S�����QEx��Q@Q@Q@�?�t���O�^�:W���0�������ht�K��������(���<��(��(�C������������<[�!H����k��C��aQE��QEQEQEVn���?��*���'�������}��>��*QE�g�!EPEPEP]������q�������� ����?�]8_������e�����(�\����(��(�M����?��L��C���5������JZE���K^���EP0��(��(����w�t�����������/Cl?����3�(��?l
(��
(��
(��
����j���c���T�zO�����?�UG�Fu>	z�EW�~xQEQEQEQEQEQEQEQEQEQE!����|��{�_�2^�5�I=?�o�?�/x���K����TQEQEQE�x��Cm�\���k\W���6����ua�pc���2(���(��(��(��J������j������6���3��W��_��!��+�O���(��(��(��;��*�����3^c^���l��������g_����f�Q^��aEPEPY����������k;_������U�����o�K��%Q^��!EPEPEPP]����J�����Y>���K��GV������d�E���EPEPEPZ^�������k6��;�#-��v�������#�3��������Q^���QEQEQE����C�\W�������a���3X����>o�iz�QZ�Q@Q@Q@���|�j�Z�?���Uk�q��3�?\���_K�(���=`��(��(��(���"������������F�����8���y���0��+S�(��(����������k����	w�\�p5�a>x����QE�y�EPEPEP7?�����@�Z�?��'���+���G���?��z��EW�}�QEQEQEC���O����u�C�y.��#��_	����J��>[;�,=�(���
(��
(��
�<[�!H����k��C����������p��8���fQ^���Q@Q@Q@f�?�������G�r��^ng���G�����c��"�Q_&~�QEQEQE�|?��m��w�������
�����A���*<����_/��h�����
(��
(��
d����������?���S[��y������_�>����(QE
(��
(��
�����*9��?�����K���hz��������S����(��(��(�zO�����?�UJ����j���c���T~$gS�����QE{���EPEPEPEPEPEPEPEPEPEP��y����E�c%��Z�����6�#�"����d��k@�EPEPEP\W���6����v��x��Cm�\���V��?�_3"�(�P���(��(��(������z������z��o�/_�>���%���(��d�(��(��(��+��!�"����C5�5����V���������&u���Fm�E�&QEQE����
����+F����]��E]?�z�V���QE��2QEQEQE��z����.���O�c_�R�����z��%��4QE|9�@QEQEQEV���e�������/���a�]��5t�5�c����z��(�tW�~|QEQEQEq�+������f�kk��_�����-{4�����^�EV�EPEPEPf�����V�����Z��w���O�2o���
(���X(��(��(���������F���_�,A���������>��c?�jz��(����
(��
(��(�����F�����]��#\
zXO��.a�E�QEvpQEQEQE
��{I��d��s���Y�o7�$}��S�����QEx��Q@Q@Q@�?�t���O�^�:W���0�������ht�K��������(���<��(��(�C������������<[�!H����k��C��aQE��QEQEQEVn���?��*���'�������}��>��*QE�g�!EPEPEP]������q�������� ����?�]8_������e�����(�\����(��(�M����?��L��C���5������JZE���K^���EP0��(��(����w�t�����������/Cl?����3�(��?l
(��
(��
(��
����j���c���T�zO�����?�UG�Fu>	z�EW�~xQEQEQEQEQEQEQEQEQEQE!����|��{�_�2^�5�I=?�o�?�/x���K����TQEQEQE�x��Cm�\���k\W���6����ua�pc���2(���(��(��(��J������j������6����|+��/����E�'��EPEPEP^���l����1�N������?��f����3��u��3n�(�P�0��(��(���@W_�QZ5����
����*��k��������(�l����(��(��(�.���O�OP]����J���e��#���O�K�FMQ_~�QEQEQE����l?���F�kK���2��a��]?�z��?�?G��:
(���Q@Q@Q@o���0�����Z��_������5�^�����?���QE��QEQEQE�����v���C�>��V�7��?S����E���(�3�
(��
(��
(��=+�_�,A��������W���'��k~��?����X������
(��9B�(��(�
:���.������o�@����^��g��zQ]��QEQEQECs���Y�s���Y�y��#�~�����=@��+�>�(��(��(�����������:�����C���O����u�C�zX?��-���EWq��Q@Q@r-���?���f5��!���
C�\�c]8_�X���
�(�T�B�(��(��(��u����iVn���?��73�v#�xs�F1��R�(��?Q
(��
(��
(��
�>��6�����W]������q������^m��/��u�QEz��Q@Q@2o����T�d���������<�~��R�/�JZ�O�
(���Q@Q@G?�{��������w�t�u>	za��U���QE|)�`QEQEQEU�'�CV?����*�[��5c�_1��B�?3��K���(�������(��(��(��(��(��(��(��(��(����
_O��
xA-���e��wGk
��
�����y�C���$�����k�������\v��KI���/O��r��?jM7��t�3�������B����G�����O�����g��^m�G�E���{���'�x������mJj����Q��-�J���N��(���3[/�@:�����?�R���?���~'�K������%����=c���A/�������4���	x?���G��3��K����z����O�!���K�k'���%����=g�xc�����>��=�B��L:~�
����\U)U�����g��	xW�������g��	xW��M���o���g�+y�VG�!���%�_��7�G�!���%�_��7�G����+y�VG�!���%�_��7�G�!���%�_��7�G����+y�VG�!���%�_��7�G�!���%�_��7�G����+y��i���j/$+�o��X��s��V��!���%�_��7�YV?
�%i�j�>������.d�&?;��|�j���X�j0�{�'�!���V��EG�
�����1�|�����
�����1�|������J��la���J*?�W?�%����o�&��W�?�%����o�&��T��7��IEG�
�����1�|�����
�����1�|�����J�������(���]�P����?�����?�]�P����?�����>�P?�0�q%zw�?�U�����f������	xc�������
��c����.<����s��]z�+��s}M%^����T������%����=c���A/�������K������%����=c���A/�������4���	x?���G��3��K����z������u�������?���~'�����wv����v8��3��UA�I�*E�+�$��?����A/
������?����A/
���������L�~�[����?����A/
������?����A/
������>�L>�[����?����A/
������?����A/
������>�L>�[����?����A/
������?����A/
������>�L>�[���?����q+1��r��`H�s��j_�C>+��K����o�&�k
>&��Ik{�xdF��8��e;��/������!(8�����N�f�M2�J*?�W�?�%����o�&��W�?�%����o�&�k����0�q%�+�������7�G�+�������7�G�*����$����w�C��^��f��h��w�C��^��&��h��@���y���T���(�K�����M���(�K�����MT��o?����;�#-��v��O�+�������7�SZx����W0�^�"m��&#?��Tp��3��a�NQW�>������O��g��������}��?���~g�L�C���4���	x?���G��3��K����z��+�>�����^����?��������=.��O��g��������}��?���~g�
���a���3X�F��_u�5���=�B��L8������_��^��s�5�S�B0I�=lY�rV����?����A/
������?����A/
��������T���[����?����A/
������?����A/
������>�L>�[����?����A/
������?����A/
������>�L>�[������C>+��K����o�&��C>+��K����o�&��S�V�3���������.����1%�����n�G���\Z���������/���o��F�8�?��>(�K�����M|�&������>�.�(����R�H�������	xc����������	xc�������T;��0�q%�+�������7�G�+�������7�G�*����$����w�C��^��f��h��w�C��^��f��h��@���y���T���(�K�����M���(�K�����MT��o?��������F�������f�������$��NO'>�k���%����=zT��������iN;6��tW�}��?���~'����%����=Y��tW�}��?���~'����%����=z]��c���A/�������	x?���@��� K���k��\i�.m��]K�$]����Y��|W��������Mva�F�Z��b���4�k�Y��|W��������M��|W��������Mt}j���
�F���g��	xW�������g��	xW��M�����a�
�F���g��	xW�������g��	xW��������a�
�F���g��	xW�������g��	xW��������a�
�D~-�B������0�(�;>ys����zd{�}g����wMk�K�>K:H|��NQ�v��R�����A/�3�5��x��C��yx`)N����T���(�K�����M���(�K�����My�T�}����$����w�C��^��f��h��w�C��^��f��h��@���y���T���(�K�����M���(�K�����MT��o?��������	xc����������	xc���������c
��z��:�|'��Z+����N��+��/����Rb2?�5��?��������0��N-H�3,U<L�*}�tW�}��?���~'����%����=t�Y�tW�}��?���~g����%����=z]��c���A/�������	x?���@�\���)�q���a���?���~g�����L��j>���m�a�~���N�����Z|�/QY��|W��������M��|W��������Mw}j���B���Ed��_��^��s�4��_��^��s�4}j�}B���Ed��_��^��s�4��_��^��s�4}j�}B���Ed��_��^��s�4��_��^��s�4}j�}B���\����x�Q��T��	@�����V��!���%�_��7�Y��w�����A�xk�q��M�\
�u5����\!��e�����]��.�Q������A/�3�4�����A/�3�5�}R����o?��������	xc����������	xc���������c
��QQ������A/�3�4�����A/�3�4}R�la���J*?�W?�%����o�&��W?�%����o�&��T��7��I]������q������W?�%����o�&���
�]���+=K�{dm��)�8���mG8O��X���z�/}Z������%����=c���A/����������4���	x?��OG��3��K����z��+�>�����^����?��������=.�7���t�*���g���������e������?���'���G����>+���xW�������g��	xW���������g��
�F���g��	xW��M����g��	xW��������a�
�F���g��	xW��M����g��	xW��������a�
�F���g��	xW�������g��	xW��M�����a�
�F�cx�W}����P��\F�T���������A/
���������5����3i����<�����en0���D�4���,XT���h�(���]�P����?�����?�]�P����?��������T?H���y���T���(�K�����M���(�K�����MT��o?��������	xc����������	xc���������c
��QQ������A/�3�4�����A/�3�4}R�la���J����j���c���P��w�C��^��f��jK|S���x�/����Rb2�Maj'rg�������h�4���	x?��OG��3��K����z��=.��O��g��������}��?���~'�K������%����=c���A/�������4���	x?���G��3��K����z��+;@M]4+e�$��jaO��0���m
�L~9�(��(��(��(��(���|��OW�4��R�}Ff���|����D�r3��~��<-���I������i:<��;9��s0����(���:(QEQEQEQEQEQEQEQEQEQEQEQEQEQ\_�<oy�kV����5_^DfX�}��D<��=�Z�(�?C�84�.����i+YD���}�z������(��(��(��(��(��(��(��(��(��(��(��(��(����k6���5mI�Z���J�d����������W}g�N�Z��K��y���8BA�8�C��h��(��(��(��(��(��(��(��(��(��(��(��(�W������/�&s�=���on���po�1=t^��M'L�����z�_�hV"����m�EQEQEQEQEQEQEQEQEQEQEQEQEQ��y����y[lq�wo@I�2��?�H���'Q��2�k��7h��w
�(�h�F���en���O��(��(��(��(��(��(��(��(��(��(��(��(��(���<i�K�U����t����IHc��Ek�Y���Z���5�����j�����1ko�,#��C<��7���(��(��(��(��(��(��(��(��(��(��(��(����f���
��~�[Y��HTd���@����o��\�x�P�m�?��Y����M$�G�!#<t���D`�
��� �4�(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
+��_��l��R�w����M�_2yOo�}��p)-<[sa��5�i��f&�2����!Fw��u9�w�>8��m�[�h^�dhT!!�)b3��w$W��&��|��k���G��d��A}�8&�|u�X�e������ G���+*���?Z�ou����H�q����	�CT&u��L���������<|}.���x�]�{?%�XD�H������s�V��O��2/�'���gn�RE��z�dxG�z��<S�����=A
���(_����s����9�^������u����b���n��y+�q�n��Vm;YB�js�z� ~5�xk����/{��+9.u1���L��7O���EPEPEPEPEPEPEPEPEPEPEOT��4M>K�Z�;X�^i�*������W���S�w�!��oB���b'�o���Ju<�5j����4���j�r��K��^@�C)=�C����4�M2�����X��@95�M��_�%�v?�����5��y���eX��;I9�1V>���|I����=�]Ail|����I�9�����dg"�<\�w�[{�s�`�?��Y��]OT��_���7W6��������'�@���J�
����x���>�j�c��-�"��'�+g�w����6�V����<�{"_3��2O#�����/�Yj�,������Ingf��
�H�(��*f>%�L�����W��L�nY��|���W8�9�V�����������5����UG��K@Q@Q@Q@Q@Q@Q@Q@Q@Q@f��|N���D����!���abr#>�?D������O�h��sD�K��+���QBY��h�b������k��#|M�|U�������-�����>a��4�p+�8�����t�Q��r��9Z	#L�Y!v,p��?Q����h����~���5h	l<�2Q[�`;�������.��I��M���U*��;cU����O�!��V�����r�/�����2�����>�M��������T�2�b���|������^���i�=SR�|q-�mi �i�p,_�*����~�5���M�-<G��T��4r� _���r;WK��xs�h���"�������a��$�t�n%���P�yQ�[���(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(����g�X�y��Cim��3�U���+��> k/����:@�II��u�@4q:���1�?�jx��N���^Mz�Q���ru!j�l����������=z���?�<_�I�O��f��%���5�Q��;�-��Bi��m�_���M�K�>�@�B�I�=�I�G`p0{�>?�>�����,�M�-i�6�Y~�?A@��<G���s�Z���6�vz���%���i�|���%���Z����]8�c����k����-�w�����|�\Y\����`����*�cE_�]���c���60zcv@���/�u��5���
�$�X�i$g�K�����'���W��vV�u�v��[[�6�Q UQ�O@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@W+���Z'�mn$��T�~�M�_2yo�}��q@U�i~$���������������u��/�(���q\��~(����=z����K����wyW�0_�y������W�to�f���A��(���������;�q�^�E>.�n4�{c�`��s�PN����k���m���������!���UA��>���������>mVO'Z�_����^����:E��m?F������T����d���}
r|'��g@�����,v_���i�$��D�v��������6�'�5���� f�<2��C{V��>�[����K�������M(���?
������N��|m��H��h���5v�(��(��(��(��(��(��(��(��(��(����wL�������6V���f����k��|k�x�]��@�������P�r�\�q�F:���P{Q�(��X����$�q���=��%m�	}�w�V����E����^���K�Z�w?i��[k[yM��!?)����WO����N�}~�%��5+��p�����J��1�X���4]KUO_|7��<z�j���O/�q�B���Pj����
�?x��m�-��Uo`��[������F���DZ��2���������@��
4
+O���Z��E��*��Q�b�N�1�7����P�,��M��A������1�<��9��v7��<O6����E���m��,����`pH�]40�o
Ekh6�"��=(�QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQHN:��7�
3����~���!��^��[�T�����'�f��~���i���^��!.T�>�g�[t��'��|"�t��F����>�n$2�29�RO|�!^�is��W6�,���$����FA�j=SM��4��:�!-��m�{�5�
u�k���mrR����������������������}
m2�+
WT���{�e��[$���F�M:_���!��-m|]������!�2��z�S�/�x�X���.%�tk�w1��d�q=�<����9�/��~��
S�VRM��"�n�E��������EEQEQEQEQEQEQEQEQEQEQEU{�B�K���Q�����$�8UQ�I�_����WZLz�3�:��������b8.s�z��������������[�"�{�08�0���L��O��z�Qj|wZ|����!�~e`FCQ^u7�&�77?5���FX� �+{<�1�;�9?�u>	�5
oE7W��%�0�Efhp6���WI@e�{�ki{���5���u���8������_D�5���l-����s�a�g����4��
�w��"�:YjQ/���<�3��+����5�"���B��p��9;XdP�|5������k���J��&�u�k<q���df�4hZ����|F���&�T��������>O�����
�x7���:t�K����Y���<t����d1 H��*(P]��$�4�(��(��(��(��(��(��(��(��(��(�������V��N���}k�>7����xKX������M�]'�,>������W5��o�n|/r�4?;]�n�")��,����v^0������l����'O�}�;EK�#�uQ�8����xF��]���O���t��}R��<yrOS���5�u^���R�����e����"0���@
����]6���U���5�)��# ���l�����~��qC}���Iy
���-���5o�d���6���T��n������c�I�0zg��]���}?\���OP�]"V��a|��G|��i��o
��Z�k�%������-��z����+��������Q���h��$D�`�km�v����[4PEPEPEPEPEPEPEPEPEPEPE��?�>�I�C��b� ������Mm���c��	�?js��,�l�yv��z
���7J�|5�-���7�u8oV3�	�U<���!�us?��w��T�����4I\���-}G'�����7P<71$�8�$�Xzz��s�w:��b�4<���r}��D��o,��?J���E�w���iax� d����8`3�d��<A�j�|k?��Y�16�y6��G�IL��t*O8�^��i�*�4}KD��m��Z��nl���R9b�a~�����i���.lD�>�v�wRH�,���[TGF�-4�K�����1J�\���j�PEPEPEPEPEPEPEPEPEPEPTe���5�4�o������21�FI����Xx�X�4�x����'����B��~��^=�/����Y���?�.59��<�����B����d�v��k6�xw[]|�������H�El��U�1��4�L���E����.�L�)�K���f'���W��%Aa�����������[k��HfB�#H�_G�m5��S����D%�������~<�SD���W����j�Z���]��\�l6q������?��y�|;�Y��]��4��c<�~�Z��K�;?I�Bi�-6�/V8�c�;C@Nx�@}���/��P��Ok�
�ER{���0�������u��'�TK,:�����$r���9�W�g�t���(��(��(��(��(��(��(��(��(��(��|_�jZ/��KD�MB��,v��C�����'�@"!P����\�d�
V��pq��J����,��_��#+&���YnC����������_����^��m0�zbX�� ���
r���m5������q�Ftx��_�+Ps�*1���U>kWzl���<B��KCo�gc������\d~zW�W�|]�����?����&:#�"��{c����O�}��o��=R��uXn��x�@�^
���������x_�A��Y��R�e��!�U��H�5��N�����6�D����U�0� �������i�W�b�^gy[s;��1?Z���uo�6�G�����;�`����"A�����{Wco[[� ��@� ����5%QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE������WZ}��kqyh@����@�29��u������-����a���H�����n3�L:$���~=�u�x�F���K�����n6[3�_����zO����x/M���h,uMJ[���@�-��u��MS��:]��>3�*����~�3o)��?�����I�_�x�����O��������7�u2�N?PS������s��zU��uf������b�����|#�-��������K+�?���n������~.�:w�#����V��*
fw5#�	���Mv�	�����/o�Z����I<�0f��Y�����:
����l�_���N�/��7��v�������(��(��(��(��(��(��(��(��(��(��(�//mt�I.��b���e���*�rx�C	#���!���f��:���f�z��v���y(�My��u�SB���x���^M#���yp�i�<�����j���V��kx��
'�^Y��B�?v��z3�X�(���4����������HI���H>�q��+��:�x��W����2��*>�t���Y�dRy��Q�\����2�-��5k?I��k����4��B����q���?�W�_��_xF�[����m�F�q�����@�S����_�_������4��/��0��O��2��C��~��/�����#��
&Fu��x���,Q�G2,���2��00:QEQEQEQEQEQEQEQEQEQEQEP�nn��?�]i��u��B�T�� 9�Ey�~1�qZi��4K�����"@��q)�H<?Z��Qe��q�pT�����~$�����W�����k[j�6�i���4�vT��G$��b�=/��%�W�R�D}�����L�%����0�+�C��������Z�6p���������S��!���:)�K�_�<�@��&��S�w��q���C�x[��z_�K�s�rA'�<Cl.�B�Lw���}�l�9�x���a��|�B��{�O9�����uu��xsJ��
B�M�K{�JQ-��O�g���JM�1 ���Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@^!�.��m)�z�;;e;C9������)l5M7�:	�������2�=��e����3�K�>����o4k�+J��\s����Bk�|��U���<����Sm�is���;X�����>���.�/�MoS��;�&��������k�v9�OR3���T��T��<[�	i=��0f��";|
��#'��������u[����|<�Ci?�C��c~�9��=k���=�{�>���5��E�]��O����@���<�/��@�md����P�nRN������z��8�����r�R�������H&��� 1����9<��z8����(�
Z(��(��(��(��(��(��(��(��(��(��(�5��~��/����/��7��3��z�=����i�����|+=������#��*rc=������"��Z��%����Z�H@Xf�Us��X�?�'����j�M>�������c�s��5�~�n.������z�
����|�nI-����t�f�������B����I�-m�����E�����N�������~����U��~dz���~�,F����������(��������~zV��?X��k�-'�M����}"f9�~|���g�v��g�
�V�d[]�$��g�����+����|����s\=��/p����w�@
+��yy�o-<��D�F�u�})�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q�#��1�~!h�O��m�!��K��d����0��@������9��u��1�s�����1-�t!��{�#�jH�x�R2��������"�$��O\'�|}��j����W���YJ���q!�}��I�������;!��;�gk�4��ou�q{g��im���[kC��
�����m�;��o�=*����|+}��R�M�z^.#��\��dF��<}y�w��s�i�~>�����E5�q�J	����H��R2�R��#�)h��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(��(���x���M��=�M�Ot �.K���+:��'���x�/	�.�]H��"���>�9
��#���������������S0�������(�^��/
\�Zj��{�,��~b��t�
��'����?�X����>}�������������"��/s��j���$������8S�z
�(��{��>{?[E5��6z����d��9�������xS���&k���X-<����;��x�{�ymckf�5��P4�d���L�z��S�S�EPEPEPEPEPEPEPEPEPEPEPEf�1�Y��O�^�F��c![��U��5�^���h�Kx.}�X�^�+Z�c�/�����2~����������u�Oi��8���j�'���'?N���dx��^5���4�����&�E����x�S���H�=s[Z������
����
��U����v��$L{����������[_�H%���e��FV���i>[��c�4��k�Q�,����E���<U����C��4��s���t���8
������.�i�h��V�$����H]�=I���(��(��(��(��(��(��(��(��(��(�����������+hW�I3�Q����������B�L�4q�X���pA��Ey������7��)�xu���2"��}�_��
o5s��~%�nf����&������X�j�����;����n����z-��
jW��`w98� t���nRC\��|	��Q��J�&�v-5���g�'8.y������+�<����?I>��\�qoz��F\e�\�s���b�P$�9�h�E�a��A���EQEQEQEQEQEQEQEQEQEQE&hh���2�f����O����h��*�[������[������y�;b"]?L�[��6��Y#����s�u��%Hci%uDQ�f8{�j�|C��_�����s6���-�#���G��Pja�r�y7�;Y�
��?��5���Z�1����Lz��P=k������8 AQ DE��������P� ������'���E[mn�.au���$Y2�U?����������hP�V���w.���{���h��(��(��(��(��(��(��(��(��(��(��(��(�y?�m6��:���B����!��O�onN2~����V�|Q�<���'K�_��u(��t>L}X��^�yi����q��N�9�C)"�����������V��M��+����J���3�;i?���0�����h>#�u0e���p2N?�Z�G����]�O��]�5����5���f#
�=W>��w�,����Mt�i$1$�2<O����n��[q��F����(�I8��p>�i����O(��;\Is4��R%.�D`q�U@>���(��(��(��(��(��(��(��(��(��(�������a.�f���[��$?���_k���u�x�[��|9r���l�l�~Y@��>��Y����w�)���4{�!���k�������na�rk�
�������������UQ�MS��A�x�L��v�v���TT���>%�V����[^�[�l��m���JN?�(��c ������ ��!�1�#�B��@J���:T�����
<��0����-�� ��j��+[{M
�+K���B-�=�2T6�5��(�k3������FX������EQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQESe�b��s�U,N3�+�����F2��B�����%�����=�8,���@�#��gc�PI>��Z���
>��v,���
<���v�+�Q��Ry$������\����x�A.���\������6;G�X�
.��5�>j�do;�h��tNr����@|9��x�=g���kx�Z����%�Dq*Q���k�f���;��z���j��SQ�]���H���������}�]��a�k^����g��f���2��#������!����C�O�����f�	_bOl��}�M#��uYDX]>�86��w���]�:������C�ER��4�JH��t�K�"`����P��8��``PEPEPEPEPEPEPEPEPEPEPEf�
3\������������K,v��O�$�����5/|F�4���/�w����:e�*-�Y#��=h���mt>(j��"��Kq���O�c�,����=*�����~"h��`�F��K��������&n2zW�\/���:�<&���h�!����d'�#�����O�(�-��o�xXZ�QZ��Z���[�����	s���=��q�,���|Q�Ei�t���c�L]N�9]��\c��Z����~�T���[\\0'������#`�M@4�6�G����=�X�(����02��S�[�un�]D�C"�x�P�����������H�*��US���
(��
(��
(��
(��
(��
(��
(��
(��
(���u�����
�i��i�yn��x��Z���[��s-����Y$����C���r�1��.C����x�����[w��auh�3%�?r�x�T����x���^�:}�K��~�id��sL[�;�*��O�������6�1H��iqE,$�x^��u����:���7��FM_��Y�Q������^3���3��"�}w����%����M"�#%H(T���9����#i�x���M
��d,�9b]��e	1��qZ���
�~�
+E�N�
�i4����,|�+pN+W��%��3M�m�Eq���90�6�G����	xj�j�E������!��0�t<
�������%���]h�����������������
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
3\���}sK�b_�'R���E��(2	�$?��;�w�<Y�����������h1&��=?v#�=z�PY�S�����iz~�8���5���?g
���+��<�X��3����yh�m���1�/r������<i��^=�Aev�5��0�P"�RuP�G�rx�V�������p_�	y�om�
��=Q��+��+�_��|�j�A��C�d��o\s�Mcx��������O�w��!��_���[�`mX���+��il�G��h 6��W1���O"��o��t^�U��u�r;{��B��\YZw��������.xW�'�|e,�h�r��K�[y��7��������h����[� c?ZuQEQEQEQEQEQEQEQEQEQEQI�Z��=����b��C������yX�r9������uM{��n���a�+=.�;���4����L��MQ���>���G���u���yVz��Ky{$������x?�v�-����,k$�B�:��>�5_�^'����6�5�[ZMu"Y"�Y�C�;RA<pyW��V�k�k�����N�|,�Q�@������I���_	|Q���t
7�����%��H6�v����d���������!S����Eq>�F���^"���.���Cmj�+K`�*�s���W[���~�b��M���p%�@�Oq���PP���^-�[�nQ
,��RrT�g�MEQEQEQEQEQEQEQEQEQEQTu�����Bu)��.	�����%��l����-��)�UM�L�����]�u�OIo��,�H]y��>��rO���V�E���6��f��-u{9H���s���Y�5}/P�M��/u�{�5��Y�M83}��+�������
�&�g�����d'���K�5M��y��/����N*/�q����|}!�����n_������+�����:��i��x�qog��[��#�6$������z�����DxbMh�n-,����D|����x�/�:o�m�u{���K��~�t��58�Zgr�8�8��W��^/����:g�<ql�o�����GrI ��'��/�c{��;�"i����xXB@[�=��
��@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@28c��X�FT`��Q��,4)�@�����
�O��|\���b�ht�����_��t/�^��
�&�Y���S�]F�A�[8<��q�5�~�}���M?����.��[���O�e�;d�~B/o_j���kJ����n�h�������q�by,q��#�|2��Y��|k���5��;w�a�tv;�W��EQEQEQEQEQEQEQEQEQEQEQEs�4�\^��j/k5��"�mm
��J������}CO��tyu��Ht�K���4H�����$�?^���I�1���p��k���h^��)���?��B��P�n���/������/:$S�\D�� ������<c��f����g���[�
��8�8��;T_l����
59��U�M$�U�;�'����|/�����?	��^�..e�@����3�@>�_
�{�>����j6Q���b����g$��[~�v��?�o��#�Z�+s��~`������/�����_M}s�k�}�Q�#|�tP
��������^�yocm��k����B�d��(��(��(��(��(��(��(��(��(��(��(��(�������?��� ��m^���P�jn���:�
��|#��O�	�A�j�F�Kh�����9*�v��~���|A���K����P��{�"-o�2t�y��f�g���g�Vo
��z_y���n�+�I�+{L��tm>;*�;X�*��U�EP��v���(��(��(��(��(��(��(��(��(��(��(��1\w�~���Y��^����aA�JI|�L���������Y�&��?�����D��F�Qq�@8'����X?����>��^]K�k����|�
(��$H7@$:�����f��������������4��,����g��xC�x{P�Z�u	��~�v�p�����k��/
xCJ�v��zJHL�e�����;���z���h��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��8O��*�<F��U���k%��{1[��#_�o�W?���
���^��#�M;_w��Ku�HW%�o���^�Ey�|%�T���<I��Z�V!����qn������}��@�4�O��4�>��mZ��F��}�>��#]��}��?5[I�2��� ���7+��u�5��;�����'�<l�XU�X�s ��9 �����G������Wo
���lP�3�������i�K�O������Y��\7�I�$+��o�]���5���x�����B���8�#������i��V�oa���Z�F"�5���4
�����[YA��$Q UQ��T�Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@Q@���o��k�1�%��tI!;Cc��C^g��J�-l4E�mm��O���R�0����!y�J�-wM�W�����nt��%��dg=Fx�?�8�|3~����OW�a�BS,��\���@�������g�=:��!��U���%OQ�S|?�=������{�����������QEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEW��G��A����Z����Yc�a�rO������]7����,���Th@�4�|�������z�=��#�b�R5
�UQ��N��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��
(��\/�c���j�n�t�J'Ui������+u�uE`�?�:�4�t��s,���q)����5�EQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQEQE��