blob: 62dd5308b58222a335d87ab6934cd2d6bae6a003 [file] [log] [blame]
<?xml version="1.0" encoding="utf-8"?><feed xmlns="http://www.w3.org/2005/Atom" ><generator uri="https://jekyllrb.com/" version="3.4.3">Jekyll</generator><link href="http://cassandra.apache.org/feed.xml" rel="self" type="application/atom+xml" /><link href="http://cassandra.apache.org/" rel="alternate" type="text/html" /><updated>2019-12-04T19:20:21+00:00</updated><id>http://cassandra.apache.org/</id><title type="html">Apache Cassandra Website</title><subtitle>The Apache Cassandra database is the right choice when you need scalability and high availability without compromising performance. Linear scalability and proven fault-tolerance on commodity hardware or cloud infrastructure make it the perfect platform for mission-critical data. Cassandra's support for replicating across multiple datacenters is best-in-class, providing lower latency for your users and the peace of mind of knowing that you can survive regional outages.
</subtitle><entry><title type="html">Even Higher Availability with 5x Faster Streaming in Cassandra 4.0</title><link href="http://cassandra.apache.org/blog/2019/04/09/benchmarking_streaming.html" rel="alternate" type="text/html" title="Even Higher Availability with 5x Faster Streaming in Cassandra 4.0" /><published>2019-04-09T08:00:00+00:00</published><updated>2019-04-09T08:00:00+00:00</updated><id>http://cassandra.apache.org/blog/2019/04/09/benchmarking_streaming</id><content type="html" xml:base="http://cassandra.apache.org/blog/2019/04/09/benchmarking_streaming.html">&lt;p&gt;Streaming is a process where nodes of a cluster exchange data in the form of SSTables. Streaming can kick in during many situations such as bootstrap, repair, rebuild, range movement, cluster expansion, etc. In this post, we discuss the massive performance improvements made to the streaming process in Apache Cassandra 4.0.&lt;/p&gt;
&lt;h2 id=&quot;high-availability&quot;&gt;High Availability&lt;/h2&gt;
&lt;p&gt;As we know Cassandra is a Highly Available, Eventually Consistent database. The way it maintains its legendary availability is by storing redundant copies of data in nodes known as replicas, usually running on commodity hardware. During normal operations, these replicas may end up having hardware issues causing them to fail. As a result, we need to replace them with new nodes on fresh hardware.&lt;/p&gt;
&lt;p&gt;As part of this replacement operation, the new Cassandra node streams data from the neighboring nodes that hold copies of the data belonging to this new node’s token range. Depending on the amount of data stored, this process can require substantial network bandwidth, taking some time to complete. The longer these types of operations take, the more we are exposing ourselves to loss of availability. Depending on your replication factor and consistency requirements, if another node fails during this replacement operation, ability will be impacted.&lt;/p&gt;
&lt;h2 id=&quot;increasing-availability&quot;&gt;Increasing Availability&lt;/h2&gt;
&lt;p&gt;To minimize the failure window, we want to make these operations as fast as possible. The faster the new node completes streaming its data, the faster it can serve traffic, increasing the availability of the cluster. Towards this goal, Cassandra 4.0 saw the addition of &lt;a href=&quot;https://en.wikipedia.org/wiki/Zero-copy&quot;&gt;Zero Copy&lt;/a&gt; streaming. For more details on Cassandra’s zero copy implementation, see this &lt;a href=&quot;../../../2018/08/07/faster_streaming_in_cassandra.html&quot;&gt;blog post&lt;/a&gt; and &lt;a href=&quot;https://issues.apache.org/jira/browse/CASSANDRA-14556&quot;&gt;CASSANDRA-14556&lt;/a&gt; for more information.&lt;/p&gt;
&lt;h2 id=&quot;talking-numbers&quot;&gt;Talking Numbers&lt;/h2&gt;
&lt;p&gt;To quantify the results of these improvements, we, at Netflix, measured the performance impact of streaming in 4.0 vs 3.0, using our open source &lt;a href=&quot;https://github.com/Netflix/ndbench&quot;&gt;NDBench&lt;/a&gt; benchmarking tool with the CassJavaDriverGeneric plugin. Though we knew there would be improvements, we were still amazed with the overall results of a &lt;strong&gt;five fold increase&lt;/strong&gt; in streaming performance. The test setup and operations are all detailed below.&lt;/p&gt;
&lt;h3 id=&quot;test-setup&quot;&gt;Test Setup&lt;/h3&gt;
&lt;p&gt;In our test setup, we used the following configurations:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;6-node clusters on i3.xl, i3.2xl, i3.4xl and i3.8xl EC2 instances, each on 3.0 and trunk (sha dd7ec5a2d6736b26d3c5f137388f2d0028df7a03).&lt;/li&gt;
&lt;li&gt;Table schema&lt;/li&gt;
&lt;/ul&gt;
&lt;div&gt;&lt;pre&gt;
CREATE TABLE testing.test (
key text,
column1 int,
value text,
PRIMARY KEY (key, column1)
) WITH CLUSTERING ORDER BY (column1 ASC)
AND bloom_filter_fp_chance = 0.01
AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}
AND comment = ''
AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'}
AND compression = {'enabled': 'false'}
AND crc_check_chance = 1.0
AND dclocal_read_repair_chance = 0.1
AND default_time_to_live = 0
AND gc_grace_seconds = 864000
AND max_index_interval = 2048
AND memtable_flush_period_in_ms = 0
AND min_index_interval = 128
AND read_repair_chance = 0.0
AND speculative_retry = '99PERCENTILE';
&lt;/pre&gt;&lt;/div&gt;
&lt;ul&gt;
&lt;li&gt;Data size per node: 500GB&lt;/li&gt;
&lt;li&gt;No. of tokens per node: 1 (no vnodes)&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;To trigger the streaming process we used the following steps in each of the clusters:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;terminated a node&lt;/li&gt;
&lt;li&gt;add a new node as a replacement&lt;/li&gt;
&lt;li&gt;measure the time taken to complete streaming data by the new node replacing the terminated node&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;For each cluster and version, we repeated this exercise multiple times to collect several samples.&lt;/p&gt;
&lt;p&gt;Below is the distribution of streaming times we found across the clusters
&lt;img src=&quot;/img/blog-post-benchmarking-streaming/cassandra_streaming.png&quot; alt=&quot;Benchmark results&quot; title=&quot;Benchmark results&quot; /&gt;&lt;/p&gt;
&lt;h3 id=&quot;interpreting-the-results&quot;&gt;Interpreting the Results&lt;/h3&gt;
&lt;p&gt;Based on the graph above, there are many conclusions one can draw from it. Some of them are&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;3.0 streaming times are inconsistent and show high degree of variability (fat distributions across multiple samples)&lt;/li&gt;
&lt;li&gt;3.0 streaming is highly affected by the instance type and generally looks generally CPU bound&lt;/li&gt;
&lt;li&gt;Zero Copy streaming is approximately 5x faster&lt;/li&gt;
&lt;li&gt;Zero Copy streaming time shows little variability in its performance (thin distributions across multiple samples)&lt;/li&gt;
&lt;li&gt;Zero Copy streaming performance is not CPU bound and remains consistent across instance types&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;It is clear from the performance test results that Zero Copy Streaming has a huge performance benefit over the current streaming infrastructure in Cassandra. But what does it mean in the real world? The following key points are the main take aways.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;MTTR (Mean Time to Recovery):&lt;/strong&gt; MTTR is a KPI (Key Performance Indicator) that is used to measure how quickly a system recovers from a failure. Zero Copy Streaming has a very direct impact here with a &lt;strong&gt;five fold improvement&lt;/strong&gt; on performance.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Costs:&lt;/strong&gt; Zero Copy Streaming is ~5x faster. This translates directly into cost for some organizations primarily as a result of reducing the need to maintain spare server or cloud capacity. In other situations where you’re migrating data to larger instance types or moving AZs or DCs, this means that instances that are sending data can be turned off sooner saving costs. An added cost benefit is that now you don’t have to over provision the instance. You get a similar streaming performance whether you use a i3.xl or an i3.8xl provided the bandwidth is available to the instance.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Risk Reduction:&lt;/strong&gt; There is a great reduction in the risk due to Zero Copy Streaming as well. Since a Cluster’s recovery mainly depends on the streaming speed, Cassandra clusters with failed nodes will be able to recover much more quickly (5x faster). This means the window of vulnerability is reduced significantly, in some situations down to few minutes.&lt;/p&gt;
&lt;p&gt;Finally, a benefit that we generally don’t talk about is the environmental benefit of this change. Zero Copy Streaming enables us to move data very quickly through the cluster. It objectively reduces the number and sizes of instances that are used to build Cassandra cluster. As a result not only does it reduce Cassandra’s TCO (Total Cost of Ownership), it also helps the environment by consuming fewer resources!&lt;/p&gt;</content><author><name>The Apache Cassandra Community</name></author><summary type="html">Streaming is a process where nodes of a cluster exchange data in the form of SSTables. Streaming can kick in during many situations such as bootstrap, repair, rebuild, range movement, cluster expansion, etc. In this post, we discuss the massive performance improvements made to the streaming process in Apache Cassandra 4.0.</summary></entry><entry><title type="html">Introducing Transient Replication</title><link href="http://cassandra.apache.org/blog/2018/12/03/introducing-transient-replication.html" rel="alternate" type="text/html" title="Introducing Transient Replication" /><published>2018-12-03T08:00:00+00:00</published><updated>2018-12-03T08:00:00+00:00</updated><id>http://cassandra.apache.org/blog/2018/12/03/introducing-transient-replication</id><content type="html" xml:base="http://cassandra.apache.org/blog/2018/12/03/introducing-transient-replication.html">&lt;p&gt;Transient Replication is a new experimental feature soon to be available in 4.0. When enabled, it allows for the creation of keyspaces where replication factor can be specified as a number of copies (full replicas) and temporary copies (transient replicas). Transient replicas retain the data they replicate only long enough for it to be propagated to full replicas, via incremental repair, at which point the data is deleted. Writing to transient replicas can be avoided almost entirely if monotonic reads are not required because it is possible to achieve a quorum of acknowledged writes without them.&lt;/p&gt;
&lt;p&gt;This results in a savings in disk space, CPU, and IO. By deleting data as soon as it is no longer needed, transient replicas require only a fraction of the disk space of a full replica. By not having to store the data indefinitely, the CPU and IO required for compaction is reduced, and read queries are faster as they have less data to process.&lt;/p&gt;
&lt;p&gt;So what are the benefits of not actually keeping a full copy of the data? Well, for some installations and use cases, transient replicas can be almost free if &lt;a href=&quot;https://en.wikipedia.org/wiki/Consistency_model#Monotonic_Read_Consistency&quot;&gt;monotonic reads&lt;/a&gt; are disabled. In future releases where monotonic reads are supported with Transient Replication, enabling monotonic reads would reduce the savings in CPU and IO, but even then they should still be significant.&lt;/p&gt;
&lt;p&gt;Transient Replication is designed to be transparent to applications:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;Consistency levels continue to produce the same results for queries.&lt;/li&gt;
&lt;li&gt;The number of replicas that can be lost before data loss occurs is unchanged.&lt;/li&gt;
&lt;li&gt;The number of replicas that can be unavailable before some queries start to timeout or return unavailable is unchanged (with the exception of ONE).&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;With Transient Replication, you can go from 3 replicas to 5 replicas, two of which are transient, without adding any hardware.&lt;/p&gt;
&lt;p&gt;If you are running an active-passive 2 DC setup with 3 replicas in each DC, you can make one replica in each DC transient and still have four full copies of the data in total.&lt;/p&gt;
&lt;h2 id=&quot;feature-support&quot;&gt;Feature support&lt;/h2&gt;
&lt;p&gt;Transient Replication is not intended to fully replace Cassandra’s existing approach to replication. There are features that currently don’t work with transiently replicated keyspaces and features that are unlikely ever to work with them.&lt;/p&gt;
&lt;p&gt;You can have keyspaces with and without Transient Replication enabled in the same cluster, so it is possible to use Transient Replication for just the use cases that are a good fit for the currently available functionality.&lt;/p&gt;
&lt;h3 id=&quot;currently-unsupported-but-coming&quot;&gt;Currently unsupported but coming:&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Monotonic reads&lt;/li&gt;
&lt;li&gt;Batch log&lt;/li&gt;
&lt;li&gt;LWT&lt;/li&gt;
&lt;li&gt;Counters&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id=&quot;will-never-be-supported&quot;&gt;Will never be supported:&lt;/h3&gt;
&lt;ul&gt;
&lt;li&gt;Secondary indexes&lt;/li&gt;
&lt;li&gt;Materialized views&lt;/li&gt;
&lt;/ul&gt;
&lt;h2 id=&quot;how-transient-replication-works&quot;&gt;How Transient Replication works&lt;/h2&gt;
&lt;h3 id=&quot;overview&quot;&gt;Overview&lt;/h3&gt;
&lt;p&gt;Transient replication extends Cassandra’s existing consistent hashing algorithm to designate some replicas of a point or range on the consistent hash ring as transient and some as full. The following image depicts a consistent hash ring with three replicas &lt;strong&gt;A&lt;/strong&gt;, &lt;strong&gt;B&lt;/strong&gt;, and &lt;strong&gt;C&lt;/strong&gt;. The replicas are located at tokens 5, 10, 15 respectively. A key &lt;strong&gt;&lt;em&gt;k&lt;/em&gt;&lt;/strong&gt; hashes to token 3 on the ring.&lt;/p&gt;
&lt;p&gt;&lt;img src=&quot;/img/blog-post-introducing-transient-replication/diagram-hash-ring.gif&quot; alt=&quot;A consistent hash ring without Transient Replication&quot; title=&quot;A consistent hash ring without Rransient Replication&quot; /&gt;&lt;/p&gt;
&lt;p&gt;Replicas are selected by walking the ring clockwise starting at the point on the ring the key hashes to. At RF=3, the replicas of key &lt;strong&gt;&lt;em&gt;k **&lt;/em&gt;are **A&lt;/strong&gt;, &lt;strong&gt;B&lt;/strong&gt;, &lt;strong&gt;C&lt;/strong&gt;.
With Transient Replication, the last N replicas (where N is the configured number of transient replicas) found while walking the ring are designated as transient.&lt;/p&gt;
&lt;p&gt;There are no nodes designated as transient replicas or full replicas. All nodes will fully replicate some ranges on the ring and transiently replicate others.&lt;/p&gt;
&lt;p&gt;The following image depicts a consistent hash ring at RF=3/1 (three replicas, one of which is transient). The replicas of &lt;strong&gt;&lt;em&gt;k&lt;/em&gt;&lt;/strong&gt; are still &lt;strong&gt;A&lt;/strong&gt;, &lt;strong&gt;B&lt;/strong&gt;, and &lt;strong&gt;C&lt;/strong&gt;, but &lt;strong&gt;C&lt;/strong&gt; is now transiently replicating &lt;strong&gt;&lt;em&gt;k&lt;/em&gt;&lt;/strong&gt;.&lt;/p&gt;
&lt;p&gt;&lt;img src=&quot;/img/blog-post-introducing-transient-replication/diagram-hash-ring-with-transient-replica.gif&quot; alt=&quot;A consistent hash ring with Transient Replication&quot; title=&quot;A consistent hash ring with Transient Replication&quot; /&gt;&lt;/p&gt;
&lt;p&gt;Normally all replicas of a range receive all writes for that range, as depicted in the following image.&lt;/p&gt;
&lt;p&gt;&lt;img src=&quot;/img/blog-post-introducing-transient-replication/diagram-regular-write.gif&quot; alt=&quot;Normal write behavior&quot; title=&quot;Normal write behavior&quot; /&gt;&lt;/p&gt;
&lt;p&gt;Transient replicas do not receive writes in the normal write path.&lt;/p&gt;
&lt;p&gt;&lt;img src=&quot;/img/blog-post-introducing-transient-replication/diagram-transient-write.gif&quot; alt=&quot;Transient write behavior&quot; title=&quot;Transient write behavior&quot; /&gt;&lt;/p&gt;
&lt;p&gt;If sufficient full replicas are unavailable, transient replicas will receive writes.&lt;/p&gt;
&lt;p&gt;&lt;img src=&quot;/img/blog-post-introducing-transient-replication/diagram-transient-write-down-node.gif&quot; alt=&quot;Transient write with unavailable node&quot; title=&quot;Transient write with unavailable node&quot; /&gt;&lt;/p&gt;
&lt;p&gt;This optimization, which is possible with Transient Replication, is called Cheap Quorums. This minimizes the amount of work that transient replicas have to do at write time, and reduces the amount of background compaction they will have to do.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Cheap Quorums and monotonic reads:&lt;/strong&gt; Cheap Quorums may end up being incompatible with an initial implementation of monotonic reads, and operators will be able to make a conscious trade off between performance and monotonic reads.&lt;/p&gt;
&lt;h3 id=&quot;rapid-write-protection&quot;&gt;Rapid write protection&lt;/h3&gt;
&lt;p&gt;In keyspaces utilizing Transient Replication, writes are sent to every full replica and enough transient replicas to meet the requested consistency level (to make up for unavailable full replicas). In addition, enough transient replicas are selected to reach a quorum in every datacenter, though unless the consistency level requires it, the write will be acknowledged without ensuring all have been delivered.&lt;/p&gt;
&lt;p&gt;Because not all replicas are sent the write, it’s possible that insufficient replicas will respond, causing timeouts. To prevent this, we implement rapid write protection, similar to rapid read protection, that sends writes to additional replicas if sufficient acknowledgements to meet the consistency level are not received promptly.&lt;/p&gt;
&lt;p&gt;The following animation shows rapid write protection in action.&lt;/p&gt;
&lt;p&gt;&lt;img src=&quot;/img/blog-post-introducing-transient-replication/diagram-rapid-write-protection.gif&quot; alt=&quot;Animation of rapid write protection preventing a write timeout&quot; title=&quot;Rapid write protection preventing a write timeout&quot; /&gt;&lt;/p&gt;
&lt;p&gt;Rapid write protection is configured similarly to rapid read protection using the table option &lt;code class=&quot;highlighter-rouge&quot;&gt;additional_write_policy&lt;/code&gt;. The policy determines how long to wait for acknowledgements before sending additional mutations. The default is to wait for P99 of the observed latency.&lt;/p&gt;
&lt;h3 id=&quot;incremental-repair&quot;&gt;Incremental repair&lt;/h3&gt;
&lt;p&gt;Incremental repair is used to clean up transient data at transient replicas and propagate it to full replicas.&lt;/p&gt;
&lt;p&gt;When incremental repair occurs transient replicas stream out transient data, but don’t receive any. Anti-compaction is used to separate transient and fully replicated data so that only fully replicated data is retained once incremental repair completes.&lt;/p&gt;
&lt;p&gt;The result of running an incremental repair is that all full replicas for a range are synchronized and can be used interchangeably to retrieve the repaired data set for a query.&lt;/p&gt;
&lt;h3 id=&quot;read-path&quot;&gt;Read path&lt;/h3&gt;
&lt;p&gt;Reads must always include at least one full replica and can include as many replicas (transient or full) as necessary to achieve the desired consistency level. At least one full replica is required in order to provide the data not available at transient replicas, but it doesn’t matter which full replica is picked because incremental repair synchronizes the repaired data set across full replicas.&lt;/p&gt;
&lt;p&gt;Reads at transient replicas are faster than reads at full replicas because reads at transient replicas are unlikely to return any results if monotonic reads are disabled, and they haven’t been receiving writes.&lt;/p&gt;
&lt;h2 id=&quot;creating-keyspaces-with-transient-replication&quot;&gt;Creating keyspaces with Transient Replication&lt;/h2&gt;
&lt;p&gt;Transient Replication is supported by SimpleStrategy and NetworkTopologyStrategy. When specifying the replication factor, you can specify the number of transient replicas in addition to the total number of replicas (including transient replicas). The syntax for a replication factor of 3 replicas total with one of them being transient would be “3/1”.&lt;/p&gt;
&lt;div class=&quot;highlighter-rouge&quot;&gt;&lt;pre class=&quot;highlight&quot;&gt;&lt;code&gt;ALTER KEYSPACE foo WITH REPLICATION = {'class' : 'NetworkTopologyStrategy', 'DC1' : '3/1'};
ALTER KEYSPACE foo WITH REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : '3/1'};
&lt;/code&gt;&lt;/pre&gt;
&lt;/div&gt;
&lt;p&gt;Monotonic reads are not supported with Transient Replication in 4.0, so any existing tables in the keyspace must have monotonic reads disabled by setting &lt;code class=&quot;highlighter-rouge&quot;&gt;read_repair = 'NONE'&lt;/code&gt;&lt;/p&gt;
&lt;p&gt;Once the keyspace has been altered, you will need to run incremental repair and then nodetool cleanup to ensure transient data is cleaned up.&lt;/p&gt;
&lt;h2 id=&quot;operational-matters&quot;&gt;Operational matters&lt;/h2&gt;
&lt;p&gt;Transient replication requires rolling incremental repair to be run regularly in order to move data from transient replicas to full replicas. By default transient replicas will receive 1% of writes for transiently replicated ranges due to rapid write protection. If a node is down for an extended period of time, its transient replicas will receive additional write load and that data should be cleaned up using incremental repair. Running incremental repair regularly will ensure that the size of each repair is small.&lt;/p&gt;
&lt;p&gt;It’s also a good idea to run a small number of vnodes with transient replication so that when a node goes down the load is spread out over several other nodes that transiently replicate that range. Larges numbers of vnodes are known to be problematic, so it’s best to start with a cluster that is already close to or at its maximum size so that a small number of vnodes will be sufficient. If you intend to grow the cluster in the future, you will need to be cognizant of how this will interact with the number of vnodes you select.&lt;/p&gt;
&lt;p&gt;While the odds of any data loss should multiple nodes be permanently lost remain the same with transient replication, the magnitude of potential data loss does not. With 3/1 transient replication the permanent loss of two nodes could result in the loss of the entirety of the repaired data set. If you are running a multi-DC setup with a high level of replication such as 2 DCs, with 3/1 replicas in each, then you will have 4 full copies total and the added risk of transient replication is minimal.&lt;/p&gt;
&lt;h2 id=&quot;experimental-features&quot;&gt;Experimental features&lt;/h2&gt;
&lt;p&gt;Experimental features are a relatively new idea for Apache Cassandra. Although we recently voted to make materialized views an experimental feature retroactively, Transient Replication is the first experimental feature to be introduced as such.&lt;/p&gt;
&lt;p&gt;The goal of introducing experimental features is to allow for incremental development across multiple releases. In the case of Transient Replication, we can avoid a giant code drop that heavily modifies the code base, and the associated risks with incorporating a new feature that way.&lt;/p&gt;
&lt;p&gt;What it means for a feature to be experimental doesn’t have a set definition, but for Transient Replication it’s intended to set expectations. As of 4.0, Transient Replication’s intended audience is expert operators of Cassandra with the ability to write the book on how to safely deploy Transient Replication, debug any issues that result, and if necessary contribute code back to address problems as they are discovered.&lt;/p&gt;
&lt;p&gt;It’s expected that the feature set for Transient Replication will not change in minor updates to 4.0, but eventually it should be ready for use by a wider audience.&lt;/p&gt;
&lt;h2 id=&quot;next-steps-for-transient-replication&quot;&gt;Next steps for Transient Replication&lt;/h2&gt;
&lt;p&gt;If increasing availability or saving on capacity sounds good to you, then you can help make transient replication production-ready by testing it out or even deploying it. Experience and feedback from the community is one the of the things that will drive transient replication bug fixing and development.&lt;/p&gt;</content><author><name>The Apache Cassandra Community</name></author><summary type="html">Transient Replication is a new experimental feature soon to be available in 4.0. When enabled, it allows for the creation of keyspaces where replication factor can be specified as a number of copies (full replicas) and temporary copies (transient replicas). Transient replicas retain the data they replicate only long enough for it to be propagated to full replicas, via incremental repair, at which point the data is deleted. Writing to transient replicas can be avoided almost entirely if monotonic reads are not required because it is possible to achieve a quorum of acknowledged writes without them.</summary></entry><entry><title type="html">Audit Logging in Apache Cassandra 4.0</title><link href="http://cassandra.apache.org/blog/2018/10/29/audit_logging_cassandra.html" rel="alternate" type="text/html" title="Audit Logging in Apache Cassandra 4.0" /><published>2018-10-29T07:00:00+00:00</published><updated>2018-10-29T07:00:00+00:00</updated><id>http://cassandra.apache.org/blog/2018/10/29/audit_logging_cassandra</id><content type="html" xml:base="http://cassandra.apache.org/blog/2018/10/29/audit_logging_cassandra.html">&lt;p&gt;Database audit logging is an industry standard tool for enterprises to
capture critical data change events including what data changed and who
triggered the event. These captured records can then be reviewed later
to ensure compliance with regulatory, security and operational policies.&lt;/p&gt;
&lt;p&gt;Prior to Apache Cassandra 4.0, the open source community did not have a
good way of tracking such critical database activity. With this goal in
mind, Netflix implemented
&lt;a href=&quot;https://issues.apache.org/jira/browse/CASSANDRA-12151&quot;&gt;CASSANDRA-12151&lt;/a&gt;
so that users of Cassandra would have a simple yet powerful audit
logging tool built into their database out of the box.&lt;/p&gt;
&lt;h2 id=&quot;why-are-audit-logs-important&quot;&gt;Why are Audit Logs Important?&lt;/h2&gt;
&lt;p&gt;Audit logging database activity is one of the key components for making
a database truly ready for the enterprise. Audit logging is generally
useful but enterprises frequently use it for:&lt;/p&gt;
&lt;ol&gt;
&lt;li&gt;Regulatory compliance with laws such as &lt;a href=&quot;https://en.wikipedia.org/wiki/Sarbanes%E2%80%93Oxley_Act&quot;&gt;SOX&lt;/a&gt;, &lt;a href=&quot;https://en.wikipedia.org/wiki/Payment_Card_Industry_Data_Security_Standard&quot;&gt;PCI&lt;/a&gt; and &lt;a href=&quot;https://en.wikipedia.org/wiki/General_Data_Protection_Regulation&quot;&gt;GDPR&lt;/a&gt; et al. These types of compliance are crucial for companies that are traded on public stock exchanges, hold payment information such as credit cards, or retain private user information.&lt;/li&gt;
&lt;li&gt;Security compliance. Companies often have strict rules for what data can be accessed by which employees, both to protect the privacy of users but also to limit the probability of a data breach.&lt;/li&gt;
&lt;li&gt;Debugging complex data corruption bugs such as those found in massively distributed microservice architectures like Netflix’s.&lt;/li&gt;
&lt;/ol&gt;
&lt;h2 id=&quot;why-is-audit-logging-difficult&quot;&gt;Why is Audit Logging Difficult?&lt;/h2&gt;
&lt;p&gt;Implementing a simple logger in the request (inbound/outbound) path
sounds easy, but the devil is in the details. In particular, the “fast
path” of a database, where audit logging must operate, strives to do as
little as humanly possible so that users get the fastest and most
scalable database system possible. While implementing Cassandra audit
logging, we had to ensure that the audit log infrastructure does not
take up excessive CPU or IO resources from the actual database execution
itself. However, one cannot simply optimize only for performance because
that may compromise the guarantees of the audit logging.&lt;/p&gt;
&lt;p&gt;For example, if producing an audit record would block a thread, it
should be dropped to maintain maximum performance. However, most
compliance requirements prohibit dropping records. Therefore, the key to
implementing audit logging correctly lies in allowing users to achieve
both performance &lt;em&gt;and&lt;/em&gt; reliability, or absent being able to achieve both
allow users to make an explicit trade-off through configuration.&lt;/p&gt;
&lt;hr /&gt;
&lt;h2 id=&quot;audit-logging-design-goals&quot;&gt;Audit Logging Design Goals&lt;/h2&gt;
&lt;p&gt;The design goal of the Audit log are broadly categorized into 3
different areas:&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Performance&lt;/strong&gt;: Considering the Audit Log injection points are
live in the request path, performance is an important goal in every
design decision.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Accuracy&lt;/strong&gt; : Accuracy is required by compliance and is thus a
critical goal. Audit Logging must be able to answer crucial auditor
questions like “Is every write request to the database being audited?”.
As such, accuracy cannot be compromised.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Usability &amp;amp; Extensibility&lt;/strong&gt;: The diverse Cassandra ecosystem
demands that any frequently used feature must be easily usable and
pluggable (e.g., Compaction, Compression, SeedProvider etc...), so the
Audit Log interface was designed with this context in mind from the
start.&lt;/p&gt;
&lt;h2 id=&quot;implementation&quot;&gt;Implementation&lt;/h2&gt;
&lt;p&gt;With these three design goals in mind, the
&lt;a href=&quot;https://github.com/OpenHFT&quot;&gt;OpenHFT&lt;/a&gt; libraries were an
obvious choice due to their reliability and high performance. Earlier in
&lt;a href=&quot;https://issues.apache.org/jira/browse/CASSANDRA-13983&quot;&gt;CASSANDRA-13983&lt;/a&gt;
the &lt;a href=&quot;https://github.com/OpenHFT/Chronicle-Queue&quot;&gt;chronical queue
library&lt;/a&gt; of
OpenHFT was introduced as a BinLog utility to the Apache Cassandra code
base. The performance of Full Query Logging (FQL) was excellent, but it only instrumented mutation and read query paths. It was missing a lot of critical data such as when queries failed, where they came from, and which user issued the query. The FQL was also single purpose: preferring to drop messages rather than delay the process (which makes sense for FQL but not for Audit Logging). Lastly, the FQL didn’t allow for pluggability, which would make it harder to adopt in the codebase for this feature.&lt;/p&gt;
&lt;p&gt;As shown in the architecture figure below, we were able to unify the FQL feature with the AuditLog functionality through the AuditLogManager and IAuditLogger abstractions. Using this architecture, we can support any output format: logs, files, databases, etc. By default, the BinAuditLogger implementation comes out of the box to maintain performance. Users can choose the custom audit logger implementation by dropping the jar file on Cassandra classpath and customizing with configuration options in
&lt;a href=&quot;https://github.com/apache/cassandra/blob/trunk/conf/cassandra.yaml#L1216-L1234&quot;&gt;cassandra.yaml&lt;/a&gt;
file.&lt;/p&gt;
&lt;hr /&gt;
&lt;h2 id=&quot;architecture&quot;&gt;Architecture&lt;/h2&gt;
&lt;p&gt;&lt;img src=&quot;data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAA8AAAALQCAYAAABfdxm0AACAAElEQVR42uydB7QUVda2/UcdFUUQRBRUTJhQBBOK4gCKERRUDAOKIoiKggrKCAJKDpJzzhlFUHLOIChGMOecxjBf/r51/n5P31O3qrqqu7q7cr3PWntxb3d15b7003uffQ46iBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQgghhBBCCCGEEEIIIYQQQkLFgVQIBoPBYDAYpXH00Uf/yI8IhBBCSPwQr7zyioxXX301ayxfvtwyVqxYYRkrV67MGatWrbKM1atX28aaNWtyxtq1a21j3bp1trF+/XrHsWHDhqyxcePGrLFp06a8YvPmzTljy5YtBcXWrVsdx7Zt2/KO7du3exI7duzwNHbu3JmY8PpcenUPFHI/5nO/F/qecvJ+zfdvQK6/Kbn+Jjn925btbyQi299XJ3+fs/19t/s/wcn/J3b/F9n935Xr/7wSESaEEEJIHAXYb/m1+5ATBfmNi/hGTXopufGT46BlOJ/3QKEyHFcRToIEU4AJIYSQmAqwm+LrlfxSfKMhvmEVXkpsNMQ4KjIcBxEOezY42/8TfkkwBZgQQghJmACz5NmZ/EZBfKMgvVGX3V27doUuoi7GYZfhMIgws8HuSLCVCFOACSGEkAQJcFTkN05Z3yCzvUFIbxRkN4xSG1VRjqIMe5kVjqIIx0WCnWaDKcCEEEJIQgQ4DON9k5T1jYr4hkF6KbnxEeQwCHEYssJhEuEwZ4ODGBdMASaEEEISIMBRl984ia+Xpc5RkF6KbrLEOEgZTpoIRzUb7Pe4YAowIYQQEnMB9kt+w1ryXEy5cxTE189sb5DSSzGNhxTHRYaDFOGwZoOjMi6YAkwIIYTEWICDlt+gS579yPq6Lb5hy/YGIbxhl8Hdu3f7EkmQ4jDLcBAiHKayaK+ywUFLMAWYEEIIiakAR1l+45L1jar4+i29SRHaJEhzlGQ4CSLsZTY4bOOCnUowBZgQQghJkAD71ek56VnfoMTXT+mNkvDGSXCjJshRkOE4iHAYs8FhlWAKMCGEEJIAAY76eF+vG12FVXy9zvb6Jb2UXMqx30LspQxHVYSjVhLtlQRTgAkhhJCYC3AU5DeorK9b5c5uN7cKk/iGUXrDIIuvvfaa5xF3KfZDhqMkwlHPBvspwYWKMAWYEEIIibkAJ328b9Syvl6Kr9fSG2XZ9UNm/YooS3EYs8JxFeEwlkT71RyLAkwIIYTEVIDDIr9elTwnTXzDmO2NiuzGSXDDKshRkuEkiXCh2eA4jwumABNCCCEJEmDKr7/lzkGJb1Skl5IbXzGOqwz7LcJRzgaHdVwwBZgQQghJiACHRX69KnkOU9bXjQZXXopvUNIbJ9nds2eP5xEnMQ5ChinClGArCaYAE0IIIQkQ4DjIbxSyvnES37BIb1xk1q+IghSHWYaDEGG3yqKDzgb7OS64GAmmABNCCCExF+Cwy2/QJc9hKXf2Snz9lt4wCm+cBDeMghy0ECdNhOOWDfZbginAhBBCSIwFuBD59bPTc5hLnsMqvmHM9oZFdim44RDjOMlwVEW42GxwnCWYAkwIIYTEVID9lF8vml2FPevrd4Mrt8U3ytJLYY2WFIddhr3IChcrwn6WRfvdJdrtDtH5SjAFmBBCCEmYAMd1vG9cs75hEt8ghDcOIrl3714ZlOLChTiuIhyWbLDf44L9ao5FASaEEEISLsBRHu8bh6xvUON7oyS9YRLWKETUpTiMMkwRjldzLAowIYQQklABpvwWLr9ui2+1atXUhzAGg8GwjVOrVY5ENjjs44IpwIQQQkjCBDjs8utVybPXWd9CM764Pr9vuZDBYDCyBv5WuJ0NTroEU4AJIYSQmAtwVOU3bllfCjCDwXBDgONaEu2XBFOACSGEkBgLMOU3PFlfCjCDwXBLgPMR4TCWRAcpwRRgQgghJKYC7If8+jneN+xZ33wa4lCAGQyGUwF2a8qkYkQ4ThJMASaEEEISJMCFyG8Yml0FLb9udXdW3WMpwAwGIx8BdmvuYL9LosMowRRgQgghJCECHAb59bvZVViyvubpUyjADAajEAH2OhschATnK8LFSjAFmBBCCEmAAIdZfosZ7xtU1rfY+XwpwAwGo1ABDkM2OOjmWMVIMAWYEEIIibkAR1F+w1zyXIz4hlWAN008W4zrWk0Gfv5lY+3EicaoLtXE8+2qerqNb1fXEu8sPC8jflhXi7IXYMztd7pocUPFyAlwWCXYr5LoQiWYAkwIIYTEWICTJr9hF1/Ezp07QyPAL71whjit6mHqA6EWeAzPJUmCIEBX1i7r6TbwBYP5XKs48+TDpYDH/cuHB2+tJCOo7dc+u4yY1P0Uw2PP3H9CaKsysF/4u5KrD4FXIhz1ccEUYEIIISRBAhxH+Y1y1hfiqyIMH7anPXeqOOTg/yfqX1xWrBp9phQvBH6GCOK5JEmwnwKMc798RHUtZvU+Tdx5bQX53G1XHxPr84xzHGS2FecY1yFqAqyCEly8BFOACSGEkAQJcFLlNwxZ3zAJ8L55NcThf/2LaFyvvGXGEY9dUuNIUaXSoYkph/ZTgFH2bPV8n/ZV5fPrxp1FAaYAWwpwMdngsI0LDkqCKcCEEEJIQgQ4jvIbdNY3X/ENiwCjBBUC/OkrNW2XQWYSsoIxweqxPbPPlfKCcl0EspbbppyT8dqFA08X19ctJ04+/q+y5LR7myoZ41yRab6lfnlZbn1+9SNEh7sqiy9XXGBYBvuHx/E81oV1mrPSap8+fPl8MaDDiXJZrLNV42PlY+Z9U+s899Qj5DG0vuVY+ZhZgLF/StRQMouf1fnCseCYcGzYL2TRzVJViADjcStBgxCrc4X9xj7bHRuewzKIR5ofJx9DaTXGOOvHvOIcWL0ex4lzqn9854xz5LVW1x3LWB0D9hvnEOfk8ppHiReeOEn7AkWNs61c4VB5HPg525hrPIfXYDvYtrpPcN4xltqqnF/dc+p+0t/f+B3bxPnFvuFnrF8vwFge5wzHiPOHx4P+AshKgL3OBsddginAhBBCSAIE2C/5jcp4Xz/LncMowJCEfEttIUFHHvEXKQ8QTQR+xmN4zix5kBZIF4QFmWSIkZKJZcOqyxJrCMvwzifLzKeSKyU3kBHspxobi+WwPNaNkmG9qCupgZBivyAuFcsdIiVGL95YJ9ZX7qiDxRMtjpeChiw4HsO69QKsjgNfFmB5CChej2PAclj/U/ceL48RMoXjwXaLEWCcRzyvH6OKY8W6r6lztNxfnAvsL0RSL3gHXjxfPqb2C8viNVgW50WfdcV+4tzaCTjOqXoM1wrXGBUB6rpjfdgOKgnUcjif2E8IOI4T+4DXqe3mK8AqU4zlcS/h+uNaYJ3Yvl6C8Zy65/Azjg/3nP76OxFgnCtcZ6wDx4HjwRcpYRXgsJZEh12CKcCEEEJIzAU4ifIbxqyvil27dgUuwNh+LlkzB6QM2TF9Rgw/K5lRj0E6zCWuECV9ZtEsmwhkf7EMMq9KvPC7OSuM10KAzAKMLKx+35C5Nssk1geZNWc4IWJY1kqAcTx60YQkYj166Uco4cuWMcwmwHgdZBzSBZlV5wT7Cxkzd5OGwOJ6qMfwWkifOTOM/cI2CxFgdX1xzs3XHecFsqgeQ0UBvuwwZ/mxXb2sOi2BxnLm66e+JMC29Pcv5Faf4Vb3nPnLklwl0Ob3hCpJV9cjjAJcjAgnVYIpwIQQQkiMBTiq8hvW8b7Fiq+KMAgwMnlejJ1FFi1XdhmyiixcIeWlkBSIplmArRp26SUR27KSNPUclrUSYKeNwNR+2GV39evEPuE4VOB3bB/yqxc5tbxVqTpeBzlVQozXWl1TZECRrS1EgNUxmWVfZV1xPtXv2L7VuS10DDCWQ9bZ6jl8IQDZz7UOLGOW2mwCbC7Th/jicZUpDuq9ir8vXklwMSXRbjfH8kuCKcCEEEJITAU4bvIbZMlzMVlfvfiGSYCRGSxk2iQlvCpQoqoXR5VNRYYQJanYDgRCL7vognxQyXRLkGWUzyLTZxYQjC+G7Oi3B3HTnz8lafqyXSvZUnKHzLLVsWFfrQTYnIFW60LWESKv9gvjTp0KMMQMx4HAz3gMWW3zeGqcF1VGbA58gaDOA8YI42er8dgqO1yIAEOoVfbYHCiv1h8vSoXVceB5yLDVuc5HgK3GKVtdG9xbqpxdf6+YM8WFNMGyWj4IAXYqwdlEOA7jgt2YJ5gCTAghhCRMgCm/wWR9wyTAkFPIWz6vUTIEYUX2D1KAgIiZy5khyhivCUlSwgpB1JfCQtqQ+UR5LbLGWAbLqnJTrEONfcW21fYgOcUIsNVyVplsJVnm5VA+DSnF8aBEVu0XZNWpAJuXwTlAqbO51Bbnx06AVejPgbm0WwWuWSECrL7MyLZ9fXYaX2yoLyzwxQhea+40no8A231Jo86jKvfGvYzzh/OlrgcCme84CXA+IhznkuhiJZgCTAghhCRIgOMov4WUPHvd6MpOfMMiwMjOQS71TYzMgewdJElNyQOZgOTlypzalQdje8jSZWuyBYlRWT9IDeTXqvS3EAGGfGM5yLvV9pG5dCLA2D+cF3P5NoS9UAGG+OLYIcLmLx2QxTRnxs0BETzIYrysCvM4bZxDCKp5OTVuWp1LjJ/Vi2ah0z7pM8H5CLD5fFiVwZv3WR9xFWBKcHESTAEmhBBCEiLAlF/vs765xDcsAozSUZTemrvp6p9HlhhZTv0YT7PA4rX6EmhIIWTCah5bLKdkBJlTq7G1GPOp5EhN92M1P3EhAqykGus1H7OSVycCjPVZjU3FvhYqwPrn9LKlGjlZSTukUn8OcS2xX2YxVxKrPw+qBN08tleJoDqXyO5CwK2++MA1Vg2msBxea85Aqy8d9GKOc+ykAzmWwz1nLuvGvak6Q2fLfqMB10EWja2szmcUBdgNEQ5iXHAYJJgCTAghhCRYgOMov0GVPDuV3927dwcuwEoQkEVDNhOZRogEAj/jMTynlw/II6QZ0gWBw7+QLpQv68URcgNBgTRgOYgJsqY4ZiXGamwrRASSh1AddyFn+vJf/I71YH/VPK+FCjCyhRA6iCL2H/uG40X21WkGGDKHx1EejP2GRKLcG2JdjADblULjPEAEsT1sC4Gf8Zhe5FSWHdcJP2M5nFNcC5wz/XmAmOJa4tphzmYsi6oAVbasP5dqW7iGuB9wzvBFCK6NvkQZ5xSZZnV/YJ3oEo3l9BlkXHtcA+y73Zhlde2wf9hPNAbDdjGWHF/K4Bypc4hjwe+oFsD1xTXB8jhmPG4WYHXPQt7VOsIuwPj74pUEF5MNjqoEU4AJIYSQBAhwGOXXi07PYRdfFWEQYDXmE5ICISn5UCh/xrhNc3k0JEZNTYNAeamaF1af0YOQIBuqXyfESp8FRJZSTRukloGsQNj068F6IV94HstCwiDE+vGrkGp9qbZ57Ku5kRKWU2OODyppxAVpw3L644AgWY2TVcKkjg/7h3MImdOPYbYKtU67ZfA49k0vqzhXkFCcb7XPkEKrrDCOQ39s+BnHa1V2DFlU0o6AwJrL3vUdn1WzLnXtsU/6bDPuD5wHdb3U9s2ZfnVdcb2zlc6rfYag67eN/TSLM/Yb11Etg+PCMag5q81juPG4ft5m9aWP3ZzZ5qmUghBgFZTg4iWYAkwIIYTEXICjJL9eNLsKm/yGSYD1U+VAhhG5xpui3DVbltNqnXbLQKDUMnZTIkGYsj1faEDYipnfVR2fVQm5l19YONlnHJs+65pt3K152WyBbee69up6FTt3rnmfsU6r6aDM+1foeOWwhlmAvcwGJ0WCKcCEEEJIjAU4TvLr93hfL8Q3rALMiHc4bTzFfY6GAIdRggsV4SAkmAJMCCGExFSAg5DfKI339TLraye+FGAGZZIC7IYAe1kS7XdzLK8lmAJMCCGEJFSAkyK/QZc8ZxPf1157TQYFmOFnoJEWxuxyn6MpwPi7UogEOxHhsIwL9lOCKcCEEEJIAgSY8ut9yXOurK+SXwowg8HIV4BVeCHBYRkX7JcEU4AJIYSQhApw0uQ36KwvBZjBYBQrwIVmgynBpRJMASaEEEJiLsCU38Lkt9hGV3bySwFmMBjFCHAQJdFhleBs/5fZSTAFmBBCCImxAFN+t4cm60sBZjAYbgmwlyXRcZdgCjAhhBASUwGOmvzGZbxvNvHds2ePjDgLMObrvbzmUYYGRvnMz/rligs8mfc36qHm1zVHrnmb4xy4R+pfXFY8c/8JiRVgr0qi4yzBFGBCCCEkIQKc6wNCkPP8hkF+/RDfJAjwC0+cJA7/618Mwnvy8X8V5Y46WIpwLqE58+TD5fmB3FF8S2Nc12rqg3tG4JzhC4e4f2nw4K2VZJjPi/l+i5sA4+8LJbg4CaYAE0IIIRTg2MqvV+N93ZLfuAvwaVUPE61vOdbwGAQYx3xL/fJZXwuJU1JHAbYW4GnPnSqWj6iuxazep4k7r60gn7vt6mMSN58xpL9yhUNFh7sqx1qAnUhwISXRbjfHioIEU4AJIYSQBAhw0uXXq5LnfMU37gK8avSZ8tjwr1mAL6lxpHxu4cDTLV8L4UUmr/bZZSjAWQTY7ryoLw/WjTsrUQKs5gyuUunQ2AuwVxLsdzY4aAmmABNCCCExF2DKb+Hy62bWNwkCjLGYRx7xl4xSXAgwnkMGGD9bjVu9ps7R4vzqR4hRXaxFD+XTyCyfe+oRMsuMdVnJ3ksvnCGur1tObgfr696mihw/q18Gr8PrsR6sD9nDT1+paVgGv+NxrAPrwjqxbjs5hZxhORzHsmHVxZ7Z50pZM68XXw7ot42SXvMyhQjwvnk15PNYrpDt6c8vAlKJ8dgQa1wTtdzcfqdbZlvtjnfnjHNkhhpl2ghsw+oY9OcQY8hRSq/uI2wT60amF8eBn/VjzHFdcOzbppwTewF2KsJhL4kOUoIpwIQQQkiMBZjy62/Jcy7xjbsAQxLRlMj8uBJgSBYE+YkWxxueRxnvIQf/PymmVqIHuYP8qLGukCOIJl6DMmCzJEK4IG1YFplBCJWSKcgpXod9Hd75ZDGgw4manEH4VCMu9RjWgeWw/EElJcj6fYcMHlRSfozlII4Y7wzRMx8HXottY99xDFg3toHzk0uCcwkwRBPPT+p+SsZ5NW8P50S/PawT5xfx1L3Ha+cXyyIjr8+64jpif83bx3Uw7x/ONbL6OP84zwjINbaDa6qWw/2A/YSc4zjxO+6TVo2PdSTAuF5W8h9nAQ6qJDqMEpzvHMEUYEIIISSmAkz5DU/Jsz727t0bWwG2Gv+rF2D8DAmC7EDYlLxAyNTrrEQPUo1MrDmTi8wmJE39DrmC/JozoxAmZCiVpEPIzJ2nsYwq3YbM4XclxCoa1ysvhdAsfXoZU9lQSLD+OLAuJcbmbTsZw5pNgCH32Dd9Myi77eEcmreH1+IamBtJKbkvRICxT1gn1q2vCMDPuE768eDYb2TqzevDdvXX3K4EGoFjMn+xEhcBxt8XSrB7EkwBJoQQQhImwG5NdxRH+fWq5FmJr4q4CjCydlZT0ugFGAIEicSYYDV+E/KiZNMsepAyq8yrXrpUNhEynKvRFjKb2HYhHZNVh2v1OzKUkH67cnD9ceC4IP5WmV5kXe3WYxZgnC+sWwV+x/nFuvWlyk63h/OO5ZC9Ni+HUvWK5Q4pSICR/cXv6ouObOcR2zcLcD5jgNWXH3bPxUGAKcHuSTAFmBBCCEmQAFN+gyl51stvnAUYx5VLgBGbJp4tpQfZSfyrL9s1C7ASK4gtBEcfeAzPqTLoPu2ratMCIRMM0UP5rH7MMcqCsQz2CWXLyBpi++bsMsaToiQX0qVCTdGkFzJzxllf/qs/DmRT8QWB+RgQyEjnuifUeUFWFfuOwM94DEJvHv/qdHsoOz8oy/hZZMwLEWB1Lay2jy8h9MviMfyOL0bwpQJkGOcvHwHO9lxcBDibBBczLjjuEkwBJoQQQijAkZbfKJU8m8WXGWBj516cB/OY4XwEWIUqb1YNkSCukCxkOfFaZAf15cyQPmwfcofnlFiqeYrVOGGsAyXb2CcE5MwswHbSZRZgyL6dkKoopAQa+4Asrbl8GefAyfbMWXRzqPOerwBDYu0EWIXKTiMbjww/zhGOBxUBB1lMm5XtfKNEPgkCXGw2OGoSnO3/okIlmAJMCCGEJESAKb/hkN+4jwGGeDkRYGRcsbxeXq1ETzV3ssoIOgnILmQWZbd2y0D+MF5WjYuF+Fo184IM668dsr8QaKt1KgHUZ0QhpFYdsIuZBgm/Y72Qef3j2B7KjHNtT5WYIzNud03NAgxBzSXAKtNuFvN8j1d/3ZM8BtgPCbYT4bhJMAWYEEIISYAAU379K3nOJr6vv/66jIMS2gXa/LjV+FQr0YOEoZmSVedjLK/G80I6zXMQKzlS24cIW81FjDJiJVcoETaLFrZhLlVGefVBps7LSiqRldUfB0Qfv1uNtYXkmadYMotjtiZYOKaDTF2Q89meGpNtHheNrOxBpiZYONaDLMb2otxcv3+4thBwPG71pYQSbhwnro35ixDV2Vl/biHAKFs3r08tqx8DHScBxt8VtyXY7WxwkBKc7/RIFGBCCCEk5gLshvzaCXDc5NfLrK+S3zgLMEQGmVSnAuw00wlhPahkeiPIEzK2ECOIrb5MFs9DPCFCeD2kSnUyVnMGQ8iQMUU2F+tBqPGqqtGWmoYH28B68FrVKdl87SBkyDBjO9hPrBfHi30xHwey41gWXaMhkAhIu7mBFbZvzsrmmgYJcohzr8q4c21PL8bI3uIxfHmBn7EcXqOfdsjctRrjoXG8ap2qbFm/f2pbOB6MMcb1gKybS+Uh4NgOpByvxzpxvrGc/osAnGNINfZdP2Y57vMAq78v+Uqw3yXR+Uhwtv87vJZgCjAhhBASYwH2crojym/+4ht3AUb2FcdmzsKqeWCdrAPSB4HUi5ySYNWECgE5guDpm1eh3BdjeyFJajmImT4ziiwnxExNU4TAz/qpjLBOJbZqW5AvCLJ5/CvWB5nTjzeGOFvNi6u2rbLDB5WMPTZnaVU2VS/AdudFXwptnobK6faUROrPL44D4m9VdowmZvplcX1xvFb7B+FVXxwgsC/maaMgufgiQ53vg0oamZnL3tV1wfXAfunHk1uVZcdNgLNJcFjGBYdNginAhBBCSIIE2MuOz0mRXzezvkkQYJXttRoH7FZAliB72aYxgghjGbtsqZJDtYzduiBceL6QcbsqY2u3bqzXTmZVptXN85ZreyqwjD7rmm3cLZZzOsYX6812PfTn28l+6q8jBDvXPMpxEWBKcPESTAEmhBBCEiTAlN/g5feNN96ItQAj04sMndsCF8ZA+TTkUE3DZB4Pjcxo1I8x7NMLqfmO7bpYx1GACy2JpgRTgAkhhJBECbDX8msnwHGU30LEV8lv3AUY2dJ8Sp6jHijdhfBjnCtkDGN5Ib+QMisxpgC7Gxi3bNVoKwkCnBQJdnuOYAowIYQQkgABDqrjc9Dy60Wn52LlN+4CnLRACS5kH/PQYuwxxhNjGiXVdCvqgfG15jG7jPAIMCU4fwmmABNCCCExF+C4ym+Yml05FV8KMIPByFeA8TeGEuxeZ2gKMCGEEBJjAQ6q43PU5LeY8b75yi8FmMFg5CvAxUhwoSIcVwmmABNCCCEJE2DKr/clz3biSwFmMBiFCrATCXY7G+y1BNs1TvRSginAhBBCSEwFmPIbTvlFVKxYUZvvlMFgMOzi2IrlLf/+UIILl2AKMCGEEJIQAQ5quqO4yW8hJc/62Ldvn228+eabtvHWW285irfffjtnvPPOO3nFu+++6yjee+89x7F//35X48CBA57H+++/72ngfWv1uB/HhnD7muRzPzi9x/K9d3O9F5y+rxDZ3p/Z3tdO/i7Y/U3J9iUcJbgwCaYAE0IIIQkQYMpv8PJbqPi6Kb9eiG9Q0hs1uS1GgJ1EmIU4TiIchARnE+G4S7CbnaEpwIQQQggF2NOOz0mR3yhkfeOQ7Y2T6HohwH7IcRhlOEkiHDUJzvZ/gJcSnCsLTAEmhBBCYi7AQYz7pfzmll83sr5hFt8wCG+YZddPAfZCiv2UYb9FmBIcbwmmABNCCCExFuCwya+dAHstv0E0uwoy6xtkqXNQ0uuXOH7wwQeeBd63Vo9HQYqjKMJBZoMpwcFNj0QBJoQQQmIqwFHp+BxW+Q1ivG/Ysr5hlt6oyW0xAuwkwiTEXstwGEXYq2yw282xKMEUYEIIISRxAhx2+c0lwEHKr1clz35nfYMW3yCFN2jJ9UqAvZZjv2U4bCJcbDbYCwkuNBscFQnOtzM0BZgQQgihABctwF52fKb8upP1jYr4BiG9YZZdPwXYCymOqwgHlQ0OS0l0FCS42CwwBZgQQghJkACHpelV3OS3kJJnP7O+bozx9SvbmyThDVKA3RRiP2Q4TCJMCY62BFOACSGEkIQIMOXXX/kNQ9Y3KPH1S3rDIK0ffvihaxGkALslxFEQ4TBkg/0cFxykBIdpjmAKMCGEEJIgAY7ydEeUX2/kN2jxDbvwuim2+QhwPsuHXYi9lOGwiHBQ2WBKcP7jgZUEU4AJIYQQCjDl1wf5pfiGT3qDkFw3BTgoOfZShsMqwpRg9yU4qKZYFGBCCCEk5gIc5qZXYZRfL5pdhUF+3W5u5VW2Nwmi64cA+yXGcRHhoLPBlGD/SqEpwIQQQkiMBTjM436jKL9eNbuKUtY3CtLrt7h+9NFHroQfAuylEHshw26LcJizwWGQ4HymSMo1X3tYJZgCTAghhMRUgCm/e0I/3rfYrC/F1zvhdUtq8xXgfJYPsxAHnRUOezbYr+ZYQUpwWDpDU4AJIYQQCnDkOj5HTX79KHmOqviGSXiDkFw3BdgPOfZThv0W4aCzwX6NC6YEb6EAE0IIIUkU4GLlN5sAU379K3kuJuvrtvj6Jb1xlF2vBdgLKQ6TCEclGxyWkuiwSXC+naHdLIWmABNCCCEJEOColj7bfQCLqvxGIesbFvGNkux+/PHHroTXAuy2FIdBhsMkwpTgaIwHpgATQgghMRdgym+y5dfNrK/X4hsm4XVLavMV4HyWD5MQx0WEw1gSTQl2txSaAkwIIYQkVIApv8HLb5zE10/pjbLkuinAXotxnEXYy2xwUiQ4qtMjUYAJIYSQGAtwsdnfqHR89mOqozjKb5TEN26y66UAeyXGYZFhN8cIh7EkOs4S7EZTLAowIYQQQiwF2MvSZzeaXkVFfoNodhVX8fVTeqMgvH4LsJtC7IcMhz0bTAmOZik0BZgQQghJmABHrfQ5ivJbzHjfsMuv1+KbJOENWoDdEuIkiLAXJdGU4GAkmAJMCCGEJEiAKb/Bya9XJc9xEN+wC+8nn3ziWWzYsEEcfPDB6kO5jKOPPlreg1bLh12Ioy7CURkX7LUE55pqLqwSTAEmhBBCKMCBlD5Tft2TX6+zvm6Lr5fZ3qhKbq5o1qyZKFOmjPxgfsYZZ4iOHTsWtJ4wybBXIlxsoywvs8F+l0QHKcFh6QxdSBaYAkwIIYQkRICjJL9udXyOgvx6WfLsZ9Y3KtIbpOhmywIrAS5Xrpxt9rfQiKMMhzUbHAUJtuuOH5amWF6XQlOACSGEkAQIcBhKn71sepUk+Q1b1tcL8Y278FpF/fr1ZSl0odnfoIQ4yiJcbDY4LOOCoyDBYRgPTAEmhBBCKMCxGPcbN/n1Muvrt/gGIb1eCOOnn37qeWzcuFEceeSR8p6zWybsQhxWEQ5rNpgSHEwpNAWYEEIIibkAx3ncr90HOMpvfvIblPiGSXj9kNxcsXTp0qJeHxYhdjsrHJZsMCU4HhJMASaEEEJiLMBelD5Tfu3l14tmV2HJ+ioZOemUqoaOxQxGFKLaKSd7LsJxkuAgpkfyqxSaAkwIIYQkXID9HvcbVMfnoOXXq/G+fmd9cW+NXt9LjFnfW4zd0CcVfcW4jf3E+I39xfhN/cWETQPExM0DxaTNg8TkLYNT8YKYsnWImLp1qJi2bZiM6duHixnbR4iZO0amYpSYtXO0mJ2KObvGiLm7xoq5u8eJebvHi/mvTUjFRLFgzySxMBWL9k5OxRSx+PWp4sXXp4mX3pguY8kbM8TL+2amYpZY+uZssezNOWLZW3PEK2/NFa++PU8sf3u+WP7OArHinYVi5buIRWLVu4vF6vcQL4o1+18Sa/cvEWsPLBHrDrws1h9YKta/v0xsSMXGD15JxatiE+LD5WLzhytkbPlopYytH60SWz9eLbbJWCO2f4JYK2PHp+tk7JSxXotdn20oKPTr2Fmy7h0l20rHGrkP2BfsE/ZN7Wd6v5fLY8Cx4Lg2vP+KPMb17y+Vx4xjxzlYmzofOCc4NzhHq1LnC+cM529F6jziXL6aOqc4tzjHONfLUucd5x7XANcC1wTXBtcJsXjvVHntFu6ZLK/lgtR1TV/fCfJa45rPSV372TvHyHth1s5RJffHSHmv4J7BvTN121B5L+Gewr2Fewz3Gu453Hu4B8en7kfck7g3x6Tu0dGpexX3rR/Z4CRLcFSywBRgQgghJKYCHIfS57jLbzElz0GVO2cTYCm/mwamZGSQJsBTTAI83STAs1ICPFsK8BgpQHN3jTMIMERpoRTgEvk1CTBEa4kmv7OkiGny+9Y8KWoQ4BWaAC+SQmeU37QAl8rv0pzymxbfUvlNi29afneUyK9efM0yu/uzjcb43BSm5+1l2CzCziR4YzYJ3l8iwVKAbSS4RIBxjpUEL5USXCLAqXhJJ8G4Zrh2+BID1xJfaswvkWAlwPjyA1+C4F7AlyL4ckQvwNNLvkDJJsBSglP3ohJg3KNjLAQ4lwgH0RwrKhIc9VJoCjAhhBCSEAGO87jffOXXToDDIL9hyPpmG4+ZFuDeMrMGuRiXkozxegHerARYn/0dIjN3aQHWZ39HykyfWYBLs79WAlxc9neVlv19UQreGp38SgEukV+IIQQxm/yas765xNcou5tkvJYjdmthLcNmETZmg51LMDLeSoCVBK/RJHixPGcrpQS7nwXG9ZYSvMuUBc4Q4OGaAOOemqIJ8GB5z+HLl3wFuNhssF/jgpMuwW6WQlOACSGEkIQKcBRLnym/+wMTX9XcCvfWGE2AVfa3n8flz5Nl5nBxRvmzyv7OLCL7a1X6/Epp9lfK7/I85NcovmbpLZXbzTL2fJE90suZhdgowtmzwU4leJlBgh2VQufKAksBNmaBF2lZYCXAEzPKoNNZYFUGPcqmDHqIlgWGAFuWQSsBTkU2AY5KSXSSJNjLLDAFmBBCCEmAAPuZ/aX8eiO/QWZ99Z2dlQDblz/bC3Dx5c9TtWxi/tlfk/ymIrP02TTu16n8amN9LeTXIL5mwd3iMDJlOJsI5yXBUoD1EmwshTZngVdpWeCFObPAS0xZ4NIy6ClaGfQCD8qg0wLc3zAOGPdtMVMmxU2Cve4MHeZSaAowIYQQEnMBjsO4Xy87PiddfvOd0ii3AKvy5/T4X6vyZ02As5Y/T9QE2Lr51QxNgJFxXFpg9te69Nk47lff8MpKfs0lz6VZXyvxTUvtXhlb0/GlRajnSpa1k2EpwgVIsCrpNjfFyswCL3GUBcZ5lxL85hzLMmhNgosugzYK8BSTANuVQesFuJh5g5MswWFtipVPKTQFmBBCCEmgABdb+hzVcb9xk1+vs75287bqBdhq/O+kjPG/Q0sF2MXyZ2Pzq9nG8mebsb8Zja8O5Mr+msf96hteGeXXnPVVmVq9+GrSmxLc17XYliPMUmwW4cxscH4SbFUKbd8Qy/lYYCfNsOzLoG27QW8zlUE7HAdsFuBcIhxlCbZr8MdS6HRQgAkhhJAYCzBLnym/bomvQYALGP9rVf48y5D9HVt0+bPKQioBzjf7m7302SS/qcguv9nEt1Rw3/hqe9awlOHUuvQZ4dJssFMJXpUpwSWl0FpX6AKzwGkBnmPbDCu/MujRlmXQVtMhZYwD3mgcB2wnwF6VRFOCwyvBFGBCCCEkQQKcT/a32NLnMMtvLgH2Wn69aHblZdZXxccffyw/PFqVPxciwHl1f863/FnK7wLLzs/WY38zuz5rpc+Gcb/GhldG+d1kIb9KfO2ld99XOyzDXoaVCDuX4IwpkixKoW0bYtl0hDYK8PyMZliFlEGnBbg0CzzLQRm0k/mAswkwJdhbCQ5bKTQFmBBCCImpALud/Q3DuN8oya+f4339EF8VdgJsNf/vVIMA5xj/a1n+PKng8mf3s7/mcb+mMb+GkuctmvzqM74Zwvu1ip02YS3DBhHWssHZJHidSYJtSqGzZoGtOkLnaoZlXwad2Q16QqkE5zkO2Ml8wLkEuNCS6ChKcL6docPQFMvNLDAFmBBCCEmIAMe59Nmtjs9hld+gSp714utcgEuyvzYNsErH/46yHf+7wDD+d4qlAFuVP6ezv1YCvFhKm3neX/PYX3P2V9/12W7cr77s2VDyrCt3thbfneJNh6GX4aIk2KIUWt8VOtdY4ELLoO27QZcIcCqcjANWX6DYzwc8yHYcMO5b3O9eSHChIlyIBFv9jXJLgpNSCk0BJoQQQhIgwCx9Tqb8upX1tRJgdxpg5Tf+V5XRZgiw0/Jni+ZX+WR/LUufTWN+S+XXIuubkli92L719S7x1jc5IrVMbhG2kWBTd+h8s8DmMuiszbDMcwLnHAc8NY9xwKb5gC0bYb2QtRGWEuBiJdjtbHDSJbjYUmgKMCGEEELyEuAklT7HSX79zvpmFeACG2BZjf+1FWDz+F8pv+nxv9m7P9uXP6ezv0stOz9nz/5alT4bx/xaZn4txXe3jLdtIv28jQjnlGDjFEn2pdC5xgLnaoZlKoM2jQPWyqDzHgecfT5gp42wxpU0wtILsBsiTAmObik0BZgQQgiJuQCHtfQ5zPKbS4CjKr/FiK9RgIvvAO10/t/c43/nOBr/a1/+XNr8Kp/sr3Xpc3b51YuvJrrfvibesQk8lyHDJSJsL8FbTBKcXxbYOC+wdTMs52XQSoBn2Y4DdjIfcNGNsGwEmBLsnQQHXQqdreqJAkwIIYRQgCNZ+kz59S/rq+KTTz4xCbDTDtDDjQLsqAGW/fy/Tsb/IjO50qr7c9byZ928vw6yv6Wlz1sMpc/Z5FcJbano7skRpSLsRIJLu0OnS6GzZYGtO0I7LYN2eRyweT5gFxphORHgOEqw152ho14KTQEmhBBCYizAQWZ/wyC/bpc+J1V+Ib4qrAQ43QBrYM4O0OnxvyMNDbDm5GyANTW7AOcY/5tf92freX+dZ39Lx/3ayq9JfN/9DrHXJuxE2FqC7Uuhc2WBczTDsukGXfw4YCXAk/NqhJWfAKfLoHHf4n7P9j5JsgQnqRSaAkwIIYTEVIC9anyV1HG/bs7z65f8upX1dSbAqgHW4KI7QDtpgJUx/2/e43+N3Z/ty5/T8/6aOz+bs7/60mcn8qsk9z0Zr9uEWYTtJHiHQYIdZYF18wJblUFbjQO2L4MuEWCbccDZBXhK3gJs7gQ9JaMTdOZUSEqAKcHJkmAKMCGEEEIBdr3xVbGlz5TfcI73NYuvcwG2mALJtgN0vgKcuwFW/uN/rbs/W5U/mzs/m7O/Wumzrtuznfxq4vv962L/929kBB43i7CVBGuNsXJlgQsqg37Fsgzabhyw1XzAVo2wcD3za4RVKsAZnaAdToWkF+BcIkwJjkYpdKFZYAowIYQQkiABTkrpcxLkt5CS50KzvjkFeLNJgLNMgTRw3rOi56ROzqdAsuwAPSunALd+ooVods9NmgAv37dADJvdT6x+90XD+N91+5cayp8RS3fOE6PmDLYUYHPnZ/PYX0P2VzW8spFfvfAe+H6fIba8vVZMf2mi2LZ/vbUEqzHBJVngqS+NE0s2zjNkgc1l0HoJzt4NermWBc5HgLM3wlICPKOATtCjMztBW06FNNh2KiQrAS5Ggu1EmBIc/iwwBZgQQghJsABHqfSZ8htc1hfx6aefysC9Nc5CgK3mAJ5WMk5TCfC0rcNFhePKy3UMmt+jiCmQcneAbtS0vqh5SQ2tAdZdbW+V2338uYe18b/Pj3xGHFHm8Izxv10HdZbLqvG/2cuft2aUP5tLn7PJ74Ef9pXEm4boP7KX3IcrG9ZNZ4Q1Cd5jkuBd4oWJacG7uflNFh2hrcugCx4HnHcjLJ0AmztBuzYVUu65gO0E2IuSaK8l2M3O0FEphXazIRYFmBBCCEmIALud/Y1a6bOXHZ+TIr96AR5vIcC5pkB68oV24uCD/yLKlj9KXH9nA4spkCbknALJqgO0WYAxJtUgwClhGzT1eXHh5ReICUuGaQL8jwGPy+OxF2D78b9W5c/p7O8Oy+yvanaVKb9vivdV/PhWOlI/DxjZW31QF0MmDNAk2JwF3vnhJlH5hOPSAnyHXoC3aWXQEODXso4DXp17HLC5EZYmwIsdCPDsvKdCKhXgsc4EWDcXsJ0A4z7P9j7xoyQ6KAn2ezxw2EqhKcCEEEIIBZjZ3wTIr5vi64YA17riPFHzsnPFrW1uFGWOOkJM3zIi7zmA7adAmp9VgK06QNsJcDe9AJunPzKP/81S/pyZ/S0d86uX3w9S0puOt0viLTFgVFqAL617sTi+SmWx+4PNJeOCS0uhIdct294ljj2uojirRnUpwIZmWLnGAUsBXptXIyy7TtBOp0LKT4DHORTgoZkCvNlagMMuwXZd5VkKnZ8EZ8sCU4AJIYSQBAhwkI2vKL/xkd8MAd7kXICHLXleZn879HtAjFzaR67noe73ZgjwhJUviBoXnSUGz+2ZIcBN/n6duLfDHQYBnrl+rLju1gbi6PJlxV8PO1RcdGUtMWbxoAwBHv/SUHHBpeeJRVuni0Xbpotal54vTj7tRLkfteqcn4qaYsqyMbYCbDX9ker+vP7N5eLvbe7UMrFnnHWaeKZvZ7Hvix0ZAoxxvRfVqS0OO+yv4pgK5cXdrZqLfZ/tEjc1u14MHNVHfJgSYMSAUelztHj1XFG2XFnRovWdGVng+StniIMPOVj0G/W8uPjyCzMEeNmOReLam68W5SuUk8udftapokvfJwyNsJbtWiBq17lAzFs/Tdzd5nZR6fhjU+fxr+KiurXEjJXjMwT4H4OeEGeed0bqWh4sjip7pKh37eViyopRGVMhTXxlmLiiUR1xZNky4vAyh4u611wqxi4bLO5JXb97OjQ3CPC0DSPFTXdfIyqdUFEe8/EnVhJ3PHSzmLV9jEGAz7mwuugzo4t4cvCDotYVNUSnIe0KEuBcIkwJtpfgsJZC59MQiwJMCCGEJFCAg2x85UXpcxjn+o2q/NqJb7EC3OyBG0S5ikeL6VtHyKZGyASfcd6pGQI8etkAuY3nJjyVIcDnXXy2aHhzPU2AZ28aL048tYo45tjy4oFOLUWXwR3FHQ80FUcfU1acWeN0gwC/MD2dUZ2zbpJY+to80erRu8WV11wmH7v/sRYyFm6a7kCANxkEeP2br4pTTq8m5ffJ7o+JgWN7i9bt75WCe/0tjbTyZ2Rux84aIUUUWV2UOA8e2080uPZvUoiR5e3w9CPio5/eER+mYmCJAK9/faXoM6ynfN285TM0AX7zy12i5kXnicvqXSqzzWYBXvP6MlGxUgVxTs2zRe8Rz4phUweI5q2apcdBd2+vjQNevHm2fKz6OaeLOlddLLoPeVo83bejqFqtiqhw7DHilT3zNQFu9/T9Unyb399U9B7XTXTu95g8zzj/i3dO0wR40qvDpPTi2rR75j7x1MBHxfW3NxQVK1cQF9erlbqGV2oCPGvLWHHy6VVFxeOOEfd0bC6eHfNESn5vEYcfcZhocPMVBgHGfta7qY449K+HiitvvFQ8P61zxlzAEOBJDgWYEhyuUmi/GmJRgAkhhJCYC3Ax2V8/Sp/9HvebdPktJOuL+Oyzz2QUIsBTtgyV8tv4nkZi1o70HMAd+7WV6xo4p3vBAnxzi+tlhnH8y0PSJdAlDbD6T+4u12EnwGoKJFUCvdlBCbRdB+imdzeWGda1bywzNMAaPXOoXMfIaYOlAL/x+Q5ZqoymVu9+s1dX+vy2uPWuW+SyHZ5uLz7++b2UBL9rEGAsC0k++7yzxNtfvSYFuFu/p6VkL928wFKAZ706WTz4RGuxZf9qQyfom++8SZx+5qkZAtzg+nqGEuj5G6dK2e3Y4yFNgB/r/qDoNfoZQyfoxTtnyMz7Yz3aagJ80RUXiMpVKolFO6caSqAf6na/3JZegO9+pJm8hmNS111fAt1p8MMl98ezBgFG6fyAuV21EuhiBdgvCS6kMZYbnaHjWApdbEMsCjAhhBCSMAH2q/FV1Eqf3er47LX8FtPsqlj5LVSAH+2bFp+hLz6nCfCMbaNE+ZQUX9u8fsECjMxjs1aNS8cAa3MALxS1L6/piwCXOfII8chTbS06QO8WtS+tJa5tfLUU4CkLx8p1Llw9W47/1Qvw9vc22grwhhIBXr5ticwCP9XzCbH5rdWi7NFlRbvHW2udoK1KoI2doNMC3HVA+tjMAjxkWt+MMcBnnHOauKn5tYYSaPNUSBgDXLlqJdHykeZSgBdunyLX1757m/Q4YJ0AY/z2CSdVNggwsr9XN62XMQYYgWZpLTrcahDgG//e0DAG2A0BjoMER208cJANsSjAhBBCSIwF2Ovsr5+lz1EY9xuU/HpV8qwX32IE+PzLzhEnnnaCeGb0Y6Lr6A6i25iOqXhcXPy3C0SZskeIaZtG5C3Ac7dOlMt16tfeJMDpJlh3tmnquQAv275QLjd61hBLAW7Z9m5x6hmnSAHuN+I5uaxqgKUXYJQ8Vz2paroE+ud3ZRm0PgOcnhppn3joiTYp4S4jLr/qUnHyqSeJvZ9tyyrAo1L7dfWN9cUJJx6vRfljylkKMOY9Ngswxklff+vVmgC/8vp8cX/HFnIMcOWqx2mBTLES4CGz0+d56Nw+GQKMJlh1r7nEIMBYtlKVY+W473NlnCnOvTAdKHW++tZ6BgF+8NmWnggwJTi7BEe5FJoCTAghhCREgIOc9ijo7G9Y5NdOgKNU8lysAL+wqLs2nY9dtPlHC9cF+O52t+UQ4GVFCPAmgwCPmzfCcg5gdGhWAtzXJMAHSqY9UhJ8TIVjxGMpAf6wZAywaoK1fu+KtAB/n3offLFLii8en7xwrGEuYLMADxqffv0Nza4VA8f3EuPmD0/FMHHvw3dbCvBIBwKMJmIVji0v2j11nxg49TkxeHovMWja8zITrwR4+Lx+WQXYPAYYy9aqe55o/uDNonnbJuJ2GY1l3NamsXhy0EN5C7BdEyzc806+LMpHhOMqwVEuhc6VBaYAE0IIIQkR4Kg0vqL8Biu/duJbqADf1PJqUbbckWLyxhfkFDaqBBpSgwZHF1xWQ5x2TrVSAV7aX27jmeEdMgT45DNONJRAo/Pz7a1vtpwGKXsJ9NKCBNg8DZJVCbR+GqRLUlLa8Pq/yY7N016cINc5e+lUQxYYErxy+8vyuceeelibCklNg5QW4H3pDtDfvy6mLBwnbmh6rWEuYCsBrlu/jow9phLoJ3q0LxHgdXkJ8IQlw+VyL8zonTEN0rGVK4oWJQK8eNd0mRFu+/S9GdMgLdw9RV4zcwl0o9vqO5oGyUqA8+kCrb//kyjBUSqF9jILTAEmhBBCEirAQU57FJbS5yTKbyFZX1sBzjEP8JTNQ0S5CmXFDSXjNq0E+LE+D8h19pvRVcoPJAhjP+s3ucIgwEPm9SppoFQqwDc0v1o2UJq+ZoycekcJ8MBpPW2bYM1eN1ET4EFTnk9L8dpJBgHumsc8wGiChW7LqgmWEuCJC0alM6GTBkhZfeur3TJ7e8FF58tMLjK6ENv3vnld1L/2Kp0Av2WYB3jd3uVyWTUFErpJv/759rT8puItkwBDfhEXXV5bToGkF+AdH68T55x/VqkAp45v8eZZjgR41IJBcrkxiwanBTh1XnFu+054Vj6uBBjX4LKGF6ekuIKYsX5MSoBnawLcvO0tGU2wmt53o7yGE1e9YBDgsSsGitva3CQGL+jhQICHZArwpuwCHFYJDntTrLCVQheSBaYAE0IIIQkQYL+yvyx9Lkx+3R7z65X8fv7555oAj7MQYMiHXoDb97lPLt9nVheDAM/SCfCMLSOl8F5z61WaAKMcFq+7/o6Goue4zqJNl5ZyCp3yFctlTIOE8lvI1kNd7xd9Jz0r7m53a0qojhR1/naRSYB7ZQjwy7vmiCPKHC7OrXW2nPrn5R1zDQL8QMd7ZbSR0Uq0ebyVaPv4fTKWbV8g5XL13qVyCqQqJ54guvV/SkyYP1K07Xi/zAw3uqmhzNCquYBnLp0sx/BWP/t08eSzHcQzvZ+SnZ2va9JIVD2pinj0qYfSWeFU9B9ZIsB7lmvZX/0cwPrsL0qupQA3v0kb//vYMw/JplkPd24jxs0fIQaM6yXOq32u7AAt/z5kCPCgrAK85t2XRKXjK4qTTqsqegx/SpY/39P+Dnn+q5x8vEGAp68dIx9HtvfOtk1Fu2daiYuvqi2qnnKCOP3cUw0CPH3jqNTjx8s5gFs/dbfoMa6TeLhHK1HtzBNFlWrHi+lbhjsT4NR9l68Ax1mCo1QK7VdDLAowIYQQkmAB9mvaI8pv+OXXqfiqwL011kKAJ1oI8C2tr5PztUJWlADPNAkwylxbdLhNXHjl+WLOznFSgOftmiBue6CxFGNsD2WyPcZ0Eo3/fp24p8MdmgAjuzh55XBxRaM6cioeLHt2zepi8IxeolPf9qLZPY3TApyK8S8NkWNYF26dJtYfSAvwxg9eESPnDhK16pwvjq9aWYxbNExKIDoi165TMxUXiAsRl6mopcXsFVOkAKMMes3rS0XTu5tI6cU+VDnpBPHo0+3EG19sNwgwxgIvXjdX3Nj0OpkNhvx2SokwpkVSApweH7xPTJo/Rs4XvOXttQb5xTpU9lcJMDLOLdrcKTo884gmwLs/3STue6SFnKIJ+4QGWO27PCiGzxgojwUCjAz3sl3z5bFOfWWMJr9KgG9vdYto26lVegxw6pxNWz5aXHj5BbLEGXHxlbXFmMWDRdN7bhJP9mmvCTCy8TPXj5UZeswFjGjS4joxY+OYknHcJQK8d6pYtHeKmLJ2uOwEjUww9rVM6t8Gt1whxi4fkLpHxsiKAdwz51xYXTw17OG0AG+zEuDBeQswJdidUugoZYEpwIQQQkjMBThMja/8LH2m/Lovv7YCvMlKgIdIOZGdoLMK8Fg53lNNhQQBxnjQhXsmSznSjwF+6Y0ZspFSWoBnSQHGOFM0W9KXQKenQlokVqVi9XuLpQCjbBfT92AaHyXA6AS98YNXpewp8ZNl0B+vEts+Xp0SxDU2jbCMZdB7v9wqpbN0HLCxG7RRgvfKkOXMKbFFdhcl0Tivg8b0kb/rI1N+M7O/2Ba2qcqfsS/YJ/P4XzWOWTXAwrHhGHGs+uwvxkRvSp0XnBuco/UlAmyeAgnnFucY5xrnfLlOgGUDLDkGeLa8VpgCaUnq+iEDfOOd1xgEGNfaegzwWJ0Aj5L3Du6h7AI8yFaAcc8HLcF2XeGjNh44yIZYxWaBKcCEEEJIwgQ46Y2vwjTdUZjl1yy+eQlwKnIL8GgpNzkFeK9JgFMBmcpHgFUZtF6A1+sFOBXmRlhWAmw3DnjvF1tL5t3dZtkMS0pwSlpvubOxuOPe28RbX+0qkeC0CPcc1FWWK2/YtyItvCXSq8b8WstvafYX27Ke/3eLnLJJCvBnGzX5VeN/cWzbPl6TEv7VGeXPaQF+paQDtHMBRjOy+zreLccBL9w5xTAH8MAZPeT903XE4/KLDb0AL7AQ4DkZAjwyU4C3WgnwQFsBTpoEx6EU2u0sMAWYEEIIibEARyH7G7XS57DJrxfjfe3kN7cApxthZQjwNrMAj8pbgJUEKwHWyqB1Alw6FZITAS4tg96kywIrAYYUQg4hwZDFnVnmA96bJQusJBjSOn7uSDkG+OwaZ4pHOj8o/tGrk+zoDPl96PEHDNJbKr5qzK+1/L6pk1/77K9RgHdoAlyS/bUQ4I0GAS6R3wNLDAKMDtBpAV6oK3+eL6asHCnHZR9XpZK4rXUT8eA/Womb7mokDi9zuKhd93xt/C+anOkFeL6FAKNSQC/AavyvXoBVAyy9AOO+tBPgsEpwvp2hwzge2K9S6GKywBRgQgghJEECHNXGV2Epffay47Mf8ltIybM+vvjiiywCPNAgwPqpkJQAa42wdho7QesFGBK0wCDAxjLoDAF+s0SALaZCWiUl2CjAkDgpwToB3mgpwPZl0PZZ4Gyl0OlM8KvbXxR33HurlGCMFf5bo3rihfH9NNl9NyPjax7za5Lf1Dbyy/5alD9/vCpj/K9BgA/oBbi0AzTOrcr+rjCVP8/ZPEE2wDqr5hkpET5WnHfxOeKBzi3Eoj1TTAJcUv5sFuBdJgHeYRbgYTkFGNN0oVu5lQBnE+GwSXCSSqH9yAJTgAkhhJCYCjCzv+Ec9xtF+YX4qjALsJO5gKebyqCdCfAkKUdOBdi+DHqxcRzwgWzjgDPLoNNZYCXA9llg/Vhgq1JoQzl0SUn0O4bYY4rS594ueU1u+d0mRTxb9jdX+fNmQ/mz9fjfNRblzytsx/+mS6Axdhsl7LiGSoAX6QW4RH49EeDU/WonwG5KsN1UZnGSYLdLoYPIAlOACSGEkIQIcBSyv36WPlN+nWd9LQVYzQVciACb5gKG7JQKcGkZNOQoeyOs2S6MA85VBp1PFthYCm0pwbps8Ns6Gc4I3fNvmRpe2cqvlv3dnFf2N1f5sysNsGwE2L4BVqkAY8z4TJMAT3MiwBuNAox7PY4SHHRX6KhlgSnAhBBCCAU4tNMehbn02Q35tRPgMMivWXyNAtynaAE2T4WUVyfoLI2wnI4DzlUGbd8N2ioLbF8KbTUm2CjCu0oE1yp2WYhvLvk1dn7Olv217P6cq/zZbvxvSQMslKIrAdY3wFqik9/8OkDbC7DWAbpEgLUpkDYbG2DpBTiMEuxGUyxmgZ1LMAWYEEIISYAAB5X99WraIz9Ln91oehUX+TULsJO5gHNPheRiIyyLccAr9eOALcqgc3WDtssCO5VgQzm0KRusibAKnfDqpdcq65tbfgvL/jovf7Yf/ysF+E17AXbaACtnB2iHcwCPNQlwUBKctFLosGaBKcCEEEIIBThS2d84lz6HXX71Apx9KqRcnaBHGjpBZ2+E5cY4YGfdoLNlgbWxwKaO0PpSaMN44JQEm8cEG0U4U4atQp/xLUR+rTs/W2V/7bs/5yp/LmT87+KM8b+Fd4DONQWSlQAnRYKjUgrtZxaYAkwIIYTEXIDDlv0NW+Mryq9z+f3yyy9zCHD2TtBZG2HtMgmw43HA1vMB5yqDtm2G5TgLbC6FtpJgNSY4iwhrMmyKrzKlVy++stuzQ/nVlz4XlP216P7spPxZG/8r5dfZ+F93G2BlF+A4SnCcssDFSDAFmBBCCKEAhzb7G/XSZ6/l106AC5HffJpdWcmvEuAxJgF2sxFWrjLotAAXWgad7gadrRmWPgtsNy+wfSm0jQSbSqL1Iuw09K9Ni6/q9uxAfjNKnx2M/XXY/Crf8mfn43+LbICl6wCdTYDdlOCwTI8UpAQH1RArnywwBZgQQghJoACHOfvrVeOrYkufw9Tx2U35zSW+evnNEOA8GmG5MQ44Vxl0vt2gs2WB7eYF1pdC20mwcUywUYTNGWHnsVUTX2PWNx/5NZY+55X9zbv7c2n58xLb8men43+La4CFioVsAuy1BLvRFMvPUmi/ssBel0JTgAkhhJAECjCzv8GUPsdFfvXiaxDg9b3zaIRV7Dhg6/mAc5VBl3aDztIMK+dYYGsJ1rLAFhJsboylzwabRbhUhs1SXPpYqfQaxdcovxudy6+p9Nku+2se+5t97t/iy59t5//Vjf81NMDaam6ANci2AZYSYNzzYZJgZoGDyQJTgAkhhJCECXAUs79JLH0Oo/yaBdi+EdYgYyMsu3HADucDdlwG/aa+DDr/LHBpR+hcpdBOJdhKhDfrSpeVEG/RSW6p7O7VLWctvtnlN9e4X+vGV86zv9bNr4otfy5+/G82AaYEu98QK6hpkQrtCE0BJoQQQmIqwMz+hrv0OYry+9VXX2UV4OLnA85VBp2rG3S2OYEXSnnLPhbYlAW26gqdVYLXZUpwhgibZXizSXIzhVcvvdnFNzPzaz/uN7P0WT/vr/Oxv8bs7zIt+5ur+7N1+fNcQ/lzkeN/SwR4jEmAoyzBcSyF9jsLTAEmhBBCKMDM/vpc+hxV+VUCPDolwPmOA7afD9h5GbRlN+gSCYZ0ZTbDcpIFtpoX2L4U2k6Czd2hnYlwqRBbh3E5K/E1z/NrXfacbdyvfeMrfefnYppfOS5/tpn+KPv8v9bjf8frxv/iCxuzABciwXbd2L2U4LCVQodtWqRCssAUYEIIISRBAuxm+XNQ2V+vGl+FVX7tBDgI+TUIsN044E2mccA28wHnmg4pVzfo/LPAmR2hs5dCO5dgQ3foLCKcKcMbS6Q2W2zUpNdOfK1LnjPLng3jfjPk11z6XHz217r5lXvlz07H/9oJsB8SHPVS6DhmgSnAhBBCSIIFOGnZ3zCXPkdBfksFuFeOccCFlkFbd4NeoAlwtmZYBWaBTQ2xMrtC55LgVRkSrLLB2UQ4Q4htwrx8qfTqxFfL+jqU3yzjfq0bX+nn/c3s/Gyf/c1sfpUuf57kcfmzcwF2S4JZCu19FtipAOfKAlOACSGEkIQIcJyzv36UPodl3G+Q8ouoeGxF9QGSwYhMHFvpWHmv5yvAYZHgODXECjoLTAEmhBBCEirATptfRTH7G9dxv27Lbz7iq+Lrr7/OGt98841tfPvtt1nju+++cxzff/+94/jhhx+Kih9//LGowPV55ZVXRJ8+fUTLli3FeeedJw477DBx9tlni9tvv10+vmTJEvkexL8DBgwQDzzwgLjiiitEhQoVZOBnPNa/f3+5DO7TQo7F6TnL51ogcl3bbPdFrnvK7l60u39zfeETZwlmFjh3FpgCTAghhCRAgJOe/fWz9Dmu8ptLUoqRXy/ENyjZxf0yZ84c8dxzz4kmTZqI6tWrS9mtVauWFNjBgweLNWvWyHOazzqVGLdq1UpcdNFFoly5cpZijHswrCLshQTbibDbEuxGZ+igS6H9aojlZhbYaTMsJ1lgCjAhhBCSYAF2c+ojZn/dHffrVsfnsMivW1lfr8S3GNnFe2nmzJniySefFNdee62oUqWKFNOrrrpKPPLII2Ls2LFiy5YtRUu1XeB+X7Bggcwg68X4uOOOEw0bNhQPPvigFO4VK1ZkFWO3RdgrCc43G+yHBCepFNqPLLAfZdAUYEIIIYQCnMjsb5xLn/2UX6+zvl6Ib76iif2FyEJo27VrJwUXognhveGGG6QAQ4TxfvBKdgsV4xYtWsjs85FHHplTjMMgwpRgb0uhmQWmABNCCCGxF2Bmf/1pfEX5dSa/bpY7eyG9OG6UKKtxuJBHNV4XJc0obYZc4v6wev1PP/2Ud/glxngPz549Wx7DXXfdpYnxSSedJBo1aiSz1iNHjhSrV6+W96MbIhyGccFRkOCwlULHOQtMASaEEEISLsB+TH3E7G9y5NePrK9b0otr9OKLL8pMKZpRQXIhu2hShWZVkGA0r8L5LlRu3Qq3ZVh/nsxirJp0ORHjILPBUZbgODXEClMW2EkzLAowIYQQEmMBDqL5VRSzv3GXXzsBDpv8eim+uO/QnOqZZ56RZcuQO2Q/69SpI8uahw0bJjZs2CD3NWjZ9VOKs53Hbdu2iRkzZohnn31WNG3aVBPjU089VZ7Dxx57TEyYMEGsW7dO3oNRluB85wgOYyl0FLPAQUyJRAEmhBBCEiTAQUx9FMXsb5hLn93q+OyX/LqR9c1Xerdv3y6mTp0qM5cY86qmEkKjKjyG5/B+KLZ0OU5CnE/naIjx9OnTDWJ8yCGHiDPOOENcf/31UozHjx9vK8ZRkuAwl0K73RAralngQqdEogATQgghCRZgZn+TWfpciPx6Nd63mKwvto2sLbK3GK+LbC6yuipDiWwvsr64R9war5srfv7557wjajJsdZ1wj2zatElMnjxZdO7cWTRu3FiWlCsxxu94HM9v3LhR3oeU4PA3xIpjFpgCTAghhMRUgMPQ/CpO2V/Kr78lz2bpwrlZtmyZ1tlYleNCsjB+F49jHlxcg2xyF4TgFhtByXAxcwmra4x7C8JrFmN17fB7p06dxKRJk+SXGbhHKcEfBdYQy48ssNNmWG5mgSnAhBBCCAU4kKmP/Mj+Zmv84nX2N5/S57jKrxvii2ujmjKh83L16tWlMKFrMTK9mMIHnZqx705ELkqy67UU+ynB2Zpk4f5FiTRKpR999FFZOo1MMTLGSoy7dOkiy9U3b95clATbNYBzS4KjUAodZBY4bM2wKMCEEEIIBTh0Ux8x+5sc+d25c6ccT/rEE0/IjsOYWxdz7GKuXYzXxdy7EKBC5u+Nk/C6LcRBZIOdNMjC/b527doMMcYXIDVq1BC33HKL6Nq1q5gyZYosuY6LBIepIVaYssBelEFTgAkhhJAECHAYml8Fmf11e9qjqMmvnQD7Kb9qjOiYMWPEgw8+KAUXonvCCSfI8boQYHQcxj1X6By++UqvFyL6yy+/5B1ByrCbIuyWBFuNC8Z7ZMWKFWL48OHioYceEtdcc43s5G0WY9xD+FIlDqXQzAJ7UwZNASaEEEISKMBhbn7lR/Y3rqXPYZFf7OOqVatEv379ROvWrcUFF1xgGPPZs2dPMW/ePHltipnHtxDxDUJwi40wyrBXIpxvh2i8RyHGaISmF2M0Q6tZs6Zo3ry56N69u6wygBj7LcHMAoevGRYFmBBCCKEAB9b8KizZX6+mPQpr6XMu+cU5ciq/OEcLFy4UvXr1ErfddpvW4AhZOTSr6t+/v1i6dKk89nymNgqD+AYhu15KsdsiHAYJtruPlRjj/mvTpo2oX7++qFSpkkGMMY0TxBh/2+zeU7hvw1YKHfYscJinRKIAE0IIIQkQ4ELLn4NofhWn7G8U5Rfn5/777xdly5a1lF/cAzNnzpTNiTA2U2XaLr30UtG2bVsxZMgQ2dgI28nW7Mov8Y2T8LopxGEQYT8k2GpMMP5W4AsZiDEaqikxPvroo0Xt2rVFy5YtZYUC7nPcy+XLlxfdunWTfw/cLIX2qyGWH1ngIKZEKqYZFgWYEEIISZgAx2HqI6fz/oap8ZXfpc/5yC/O1WOPPSbKlCkj/vKXv8gPiGg8NXHiRPHwww+LBg0aiAoVKshAoyo8hmltduzYkXenZz/kNynS64cMx0GCnUyPhL8rL7/8sujdu7do1aqVuOyyy+SXO3g/HHHEEVKEe/ToYRDhIEuh45IFDqIZFgWYEEIIoQCHovlV2LO/YS19LrbjM7os4wN+yYdCLU455RSZ5UW2F9kwXEsnDa/ymds3KPGNo/SaA1NHma8pgxGFwN8eL8ugKcCEEEJIjAU4Ds2vqp9+Gj8UMiIXZ1U/Xfz6668FR7ECjH34358+ZBQY33+wWxx88MGiYoVjxBV1LhLdOrUXa16cIX7+eC/Pj8eBe9fLMmgKMCGEEEIBDm3zK2R75Qf5H99nMCIVuG+LEeBixZjvm+Lit8/eED9/9BrPRUDvHS+bYVGACSGEkIQLcNjLn+UH+R/2MxiRCq8E2KkQp9837zEYkQsKMCGEEEIKEmC/y5+9mvpIfpD//h0GI1LhlwDbybB833z3DoMRucC963YzLAowIYQQQgH2vPzZreZXOJb/+fYtBiNSgfv2n//8Z8FRrAzzfcOI8nvHyzmBKcCEEEJIggW40PJnP5pfGQT4m30MRqSiWAEuVorT75s3GIzIBQWYEEIIIUULcBSbXxkE+Ou9DEakwgsBzkeI5fvmq70MRuRCCbBXZdAUYEIIIYQCHOq5f+UH+S9fYzAiFX4IcDYZ5vuGEeX3TjYBLjYLTAEmhBBCEirAUSh/VgL831/sYjAiFbhvf/vtt4KjWBlOv292MhiRCwowIYQQQnwRYL/Ln53IrybAn29nMCIVxQpwsVIs3zefbWcwIhd6AfaiDJoCTAghhMRcgKNc/qwJ8KdbGYxIhRcCnI8M833DiPJ7J5cAF5MFpgATQgghCRTgsJY/m+VXE+BPNjMYkQqvBTiXEKffN5sYjMgFBZgQQgghnguwVTlZGMqf33vvPflh5b8+3sBgRCr8FmCzCPN9w4jye8f8f00hAmwnwRRgQgghJMYCHPXyZ02AP1zHYEQqcN/+/vvveYWbIsz3DSPK7x0nAlxoFpgCTAghhFCAQyHAVuXPmgB/sIbBiFQUIsBuinH6fbOawYhcUIAJIYQQ4qoAR6n8WRPg91cyGJEKNwW4ECGW75sDKxiMyIWVABdaBk0BJoQQQhIkwHEof1YC/J8HljMYkQovBdiJDMv3zf5XGYzIBe5d/J9SiAA7kWAKMCGEEEIBDm35sybA7y1jMCIVuG//+OOPvMJNEeb7hhHl945TAS4kC0wBJoQQQhIkwG5Nf+RX+bMmwO++zGBEKgoRYLekWBPgd5aEMr7fMVusmdpb/LZ3ofbYljkDxQerJ4R2nxn+BQWYEEIIIZ4JsFvjf70qf9YE+O2XAo0PVo0Xzz58p/zXbpkNM/rKZb7YMNXTfZnc5zHDOfl511zx1AO3it0Lh2iP/WvfIvm7eV+cHAfDnXBLgAsV4vQ98mLRcdUlNbJGn8db5r3ONVN6yf37YNU47bFqVY6T96Z52X/tWyjveWzrqDKHy9dh2bZ3XCfee3WMK8foR9xx/ZUFnaskhhLgQsugc02HRAEmhBBCKMChLX9WAvwfby0KNFZPeU7uB/61W+bSmmfKZZ59+A5P92VS7/aGczJr0BPy9yYNL9Uee3/lWMv9dXIcDHfCKwF2KsNuvW+wnppnnSLuuaW+ZYx77uGC30+4T9Vj1apUynjv/LRzlrjq4hpy2RuvukgMe+YBef93bt1UVD62vDj8sEML2n4QgePD+eJ7w9k9l48A55sFpgATQgghFOC8sr9OBdgsv04FWC+/mgC/uSDQWD25Z1ocU/9aPb938Qvy+asuPldUrVxR/Pn6XM/2ZVLvRwzn5KcdM0Sfjn8Xby4Zqj32/orRlvub6zgY7oUfApxNhuU9sm9+0SG/1HnodlfWpWL1pB5pAV4+SntMCrBuO3/unSMa1jlfZn1XTHg2Yx3/3D1LNLumjlzPSyO7uLp/XoQU4Jv/Fvr9DENQgAkhhBDiigBHcfyvJsBvzA00Vk98Ni2OqX+tnm97+zWixhkniTdfSovwvMGPu7ZtrBPb/XjlaPn7pOcfynlO3n91hOX+5jqObOvDa3bP6591uX/umiE2z+gl1k/tKX7aNjXnerE+rPfrDRMc78P22X3En3tmBX5P5Aqc5z///DNreCnCbr1vpAC3u83Rsrj2uE5W97D+3lH3oX5ZKcC67aj7fNaAx2y3h/vg0vPPEKefVNlwT2D9n68da3m/YV/Mj2NZbA/bn9anvfh+82TLvwHqcdyDeA3WZ/eewPN4jdqvtABflde5x/towBMtZODc2i2H/cJ+Y/8XDu0kt4nHrN7neI9iGSyLY8ZyeEx/fOZrqvZhxfiulu9LtW94z+f7t8XunrMTYDemQ6IAE0IIIQkV4CiUP2sC/PrsQGP1xG4l4tgt47mftk6WWaphXVrJ3+vWOktcW7em5XrSH/JvzXj8niZXiasuOsfw2JsvDhIXnnuq+rAm4/6m9cW47m0yzgl+n/R8O20b+tfon8t2HFbx/ivD5H7p1wXZ2Dz9uYxlp/V5WJQrW0ZbDj9juzg2hH5ZvB7rUcsecvDB4h9tmspzg/3XL/v5mjGiYZ3zDPuAZdZP6W55bkd1bS0qli+bsU2/w4kAeynHbr1v0gJ8q6Nlnd7f6j7E/WX3WryPapx+Ys5tLhv9dDoLPKKz5ftBH9gH832B5Q7/66HyfoVM41+E+f6SMt7/UbmMWn/n+5rI1+JvgHlbd994hWH/pQA7vCe/Xj9Ou+fPOqWK9p5uUv+ijG3hXGJ/sR81z6omf8Z2sa/mewB/U7A+PI5lKlcsJ98reO+a/y5gOzfWq60ti3XjZ+yXfh/Ue3beoI7aGG037jn9/zFujwOmABNCCCEUYNcF2K3yZyXA/753ZqCxasIzcj/wr/m5UV3vlx8+v1o3Rv4+8bkH5bIHlg3NWLZalWNFtwebZTx+T5N68sO5+v27jeNF1eMqpD6sniCWj+0i/tg9Texd0E/cUK+WfNx8TvA7toufN03rIWb2S48THtyphdznz1aPynkc5sA+YH+xD+smP2vYB3zQ3bd4oLbsS8OflOu9+8a68rh/3TFFPqaOAcenlsXzeD1EYtvM57V9xu9YHttUy2Kb56Y+fGMdap8/WjFcigDWoY5LndtzSz7U45ro9y+IKFaAixXi9D0yo+jAetL3bO5lS+9v4+Pp+/ts7Xd1Hx5YNsT2tfhS5PF7bsi5zT92T5XLdr6vsWGfJz7XNmNZ7EP6Xkz/jvtObQfrwWO/7pgsml19iRTDH7dMMKwT9yder/Yb/+Lxsd1bG7aD1+FvQv/H7zIcn37b2aJR3fPlfbxt5nPaY/g7gHtevw5sH481vLRG6v06TjsfPR+5Tb5efw/gcbyPEPsWD9Aen9nvEe2Lq/R7LP34fU3/Js+Bfh/2LugrH9NfF1wz7AMex7r056yYey5fAc4nC0wBJoQQQijAno//LbT8ef/+/ekPcXumBRqrxndJf0BM/Wt+7sJzThHNr62j/f7r9onyA2Xn+27KWLbaCfiQ3zTj8XsaX5kWhJLfez/aXH6APrBssGG5P3ZNSUle1YxzIj/w92yj/Y7XWe1vtuMwR8+Hb019sD1MfLR8qOU+NLv6Yu2xmmeeLOrWqp6xjm0z02OOcXzqsTa3NZAfzr/bMMawLH7H4zhH6jEc0yEH/0W889LAjH2oetwx4h8PNDGcW5yzfYv6BX6/qGvipgDnK8PyHnltWtEhM+6pc4v70yrkfVeyrLy/2zbNWId2f5f8vmpc+j48sHSw5WvxOJ4f9Y9WjvYRr5X3mG6f9fulQgqwbjm8b3Hvmpf7bv0YeS/p14F1Xnre6RnLNrr8/PS9r3ts7LMlX4qtGWW7j3axaWq6PPylYY9nPIf16s/bP1o3kX9rsL/mZW+48gLDPbBwcAf5+7YZPTOWxXmXfxdS1wW/f/TqUPm+w/bMy+JvU8XyR2W8dmqvB12539S5pgATQgghpCgB9mr8r5fTHxkEePeUQGPVuKdLPiA+bXh82/Tulo8/1LyhlLk/dkwyPJ7+kH9LxvrvaXyFuOrCs7TfG11+nvwAa7Uvw55qkXFO5Af+Hg9ovx94eaDlftkdh1VA7O++4XLL5/p3uEMcdcRh8uevVqfHG8/s085y2boXVJfHpz8Hj7e4znJZPC4FuOT35o0ula+3Whb71vDScw3rtTtnQQTOyb/+9a+s4aUMu/W+wXogibiGVrFw8KN539/qPsR9avXa79aNks/3bn+7o33Ee63NrfVt3w8qsA/6exGvw32E/TEH9ufRuxoZ1jn4ybsz1jl3QLop3Tsv9jdsB/eu+b2v37Zd4L0FqbV67sdN6e7uU59vq723mjW82HJZLKO/B3Asp594nOWy+xb0MfxdUMeEa2s+LzgH+muHa4bff9063tX3TjYBLnYcMAWYEEIISaAAR2X8rybAuyYFGqvGdk5/QEz9q3+8za1/k5mebm1uNkTzRpfI5ef2f9iwfLUTKsrnzeu/56a6aUHQLdf53huy7ov+MfmBv/v92u8Hlgyw3F+747AKu31FLByULrH+aNkgsWtmj6zrxLEh9Ps6tmsry2WHdf673K76HefkINN4Zn2Yz5l+O0GHEwF2U44tBXjXxKID6+nWpomjZdP3TOaypfd3+vdVYzulJWpJf9vXVq54tLj7+stybvOjZekvewY/cadhn9PvB+OyUoDlPVK6XLYwL2u1zl+3jJEZ0c73Xi9/xzGl3w+dMs6Nfn12gXOQfg/YX4/+HZpr63yoeQPL5ZaPetJwD5ivgT7M+4zjzHVu1LXD/qIaw417TX+M5v9r3BwHTAEmhBBCKMChHP8L+VUC/G87xwcaq8akP0ziX/XYDxtGyCzouadVEfUuPDMj8AEe/+rXk/6Q3zhj/S1vutywbM3qJ4k2za6y3Jc5/dplnJP0h/NW2u/7X+qbsb92x2EXp59YSXRs0cjyubFd75Hr+WXzKG1bS4d3sFy2wSXnyONTv0MWera7xXLZTimJwDnSvxaBbVjFp68OMpxb/XaCjmIFuFAZ1gvwv+0YX3RIAX6gsaNl5f1tsax2f5f8vmp0+j7c/2Jf29fiNXh/fbtmqGFdU3u2luW5S4d1kL/jNVjX2wt7G/Z54rOtMvYD+yDvkZLfcS92ue9Gx+fBap2Ijn9vJCXw961j5f6cWe14y3Oj37Zd9H6kmSh3VBnL535Yn662wDnA73hvNGtwoeWyQzvdZbgH1D5aLbt1aklvgNR1we8zeqUb7X254oWc+4vjle9ZF+41/bkuRICdZoEpwIQQQggFOLTjfzUB3j420Fg1+vGSD4iPa4+NfOpu+UH802UDLF8z9pmW8jX75vbQHsMH4zZN62Us26xBbVGv9pna7y1vvEx+qPx9y+iMZfGc+ZykP5zfq/2+f3HvjP21Ow67uOu6S6Xc2z0nP+SX/F6x3FHiodvqZyz35YpBMkOOfVaPNbrsXHHh2dUs14vtyQ/TJb93/Ps18osEq/NgDikYuu0EHW4KcCEy7Nb7Ji3ANzlaFtfP6ho0uORsw/2t7kPcp/rrp98O3jd4fzW/5uKM9eFeK3fUEeLFwY9IScb7R/88HrPaZ3ypo9+/JlddIC6tcarj86B/j+kD+yq/BBr6qNxG/0dvLfj+XDsunR1fPqJjxnPYvpT9Bc/L33GMeH99uKRfxrI4Lv09gH2zWy/Op/7vAtaH3/E3LNf+Yh/071m37jkKMCGEEEICF+Agxv9qArxtdKCxalTH9AfE1L/qsZrVT0x/8LZ5zQ9rh8gP4h3vvlp7DB+A8cH902X9tMf2zOwqP8TWq11de2zjhPSH4C6trjesc+mQ9lIKzOdEfjjvdo/2+y8bhsnlerZtkvM47GLt2CfS5ZaPNjM8/uKgh+S6hz55h/YYtoPHZjzf2nD8N9Q9Tz4uP/jrjsF8bL9vHikevaOBXFZ+mC55fN+c7vKxbq1vNOwDzl+jOueKqT1aaY9pghHwvaK/Jv/2b/+WNbyU4fQ9MqrokAIsz3/uZXGv415eO/Zx7TFcI6wjfX+nH1s1Kt2Qaf+i57XHqp1QIWM7I59KZzFxH709v4f2+C8bhsrl8dyZ1Sqn7oe+htc1a1BLfimD95Z6rGfbdKa45Y11tMewn3gMz/2+eUTGdvE+1J+Hid1a2h47jg/7kv5SrG/G89hf/bazBdZVucLRhv3HvuCYml9zkfbYl8v7y8curXGKdn6+XTVI3Nekrvw7Y74HsBzWi/OvzuOgjrfJx9J/FzoYziEe3zmti/YY1l235ulyPeoxXDMcmxv3mv5c5xLgYsYBU4AJIYSQmAtwLvkNQoCdTH9kEOCtIwKNVSMfS39ATP2L3zeOT8vh0iEPZ31dm6ZXpD6gHil+Wf+C/H3/op7yQyXivsaXi+ZXXyg/qDaqc07qQ+8Zhtd2a31DuvNs6sMm1nND3RpyWfxsPifyw3nXFobHWt5QR34Yb3DxWWLtmI6G48D2qx1fwTIGdbhVWwd+Nu+DzMql9vv3TcO05fBzs/q10g2TqleVx4LjxnL4Gfui37eebW9Kzyl84rHyeWwXr3v0jvryZ/2yQ5+4XduHjnc3lOcN+w/Z+HblAG05vM68nSDDiQC7Jca2ArxlRNEhBfj+Gxwt++2KAeLcU0+Qr7nw7JPFmSdX1q6LvL9Llls1In0f7l/YU3sMy1ltZ8Zz92mCVrVSeblefLGE3yHbuO/Mr8F6sSzuVdw3WDf2q0m989P3iG5Z3F9YDttQ9yJ+792uScZ5kO8xm2PHfmIZq/1Rx3dQljG18r4vWfbTl3vL/VbvJwR+xnv5h9WDDOvdOO4Jeazqfa2WU+8b/bL69eL9ifOHa7T0hfQ8wLguallsB+dD7UPdmqfJ84793De7q7Ycrpl+39265/B/iRfjgCnAhBBCCAU41ON/lQD/a/OwQOO9Bd1F1/uvl//i91eHPix6tLlR/LZhSNbXvTHrGcPr1LraN/+bqFcLYnipeG3602LliPZiwjN/z3g9tnPXtRfLZdvdWk+8Pbeb2DGls1ynfjn8jsf1j2Hfxjx9p9zW2tGPGY4jW2Bf9OvBa+9rfJncB+zL/L4P2B7v4gFtRce7Gsht4mc8po7TvKw6DhwX9vPntYPl7/gwbbUslsO6IDFDH79NLq9fZtBjzbLum99RjAAXKsRmAXbjOHA/6O/fXIHrgnsZ1xLX9Zvl/eTr9fcnHsN69ddww7jHbbeD5eb0uk+7R3Gt8b7ZMvFJcWRKyvBeNL8G2xjRqblcfsqzLeU68H5EWL2/sU61bqv9wP5+sqRX1vc6zvnLg9tZPo/jwzrsAs+b379YF/YJx4e/BdnOOZ7HeVfrwc929wCWwfNYP7aDvytY1vze1/+tw37M6HlvxvsO58q87268d9wSYCsJpgATQgghFGDPBLjY8mdNgDcNYUQ0Tqt6rGh/+1WOlr2r0UXi8vNPjcVxuynA+cqwJsCbXoh9zO97v7jvpjrik5d6BrofHe9MVy/4uU0c88rhD4sfVvbNeK797fUM+/PzmgFy2Q8Wdc9Ydkq3tCx/sfT5UFzTYgTYSRaYAkwIIYRQgPOWX98FeOMgRoijyz0Y+3mIeHXIg4bH+z2cHne5ZXwH7bHF/dNTrOA5/bIrh7WT6xjasWkszolXAuxUhOX7ZsMghg/x86p+sqS4V9sbfd3uJ4u7y2ERDS6qLt6e1UU+9tvaAWJG9xbyvTSo/c3asni85hlVxJknHyc2jH5Uexw/V65QVjT92/mhOZ8UYEIIIYSEToD9Gv+rCfD6AYwQx29r+om7rqmdHi+Y+pBd74LTxGlVK8rfB7VvkrF811bXyHGWWAbLXnjWiXLZ2xtcINcVh3PitQDnEmFs/8/1/Rk+xPTu6Y7wHy/q6vu2N49tn5LaSvJ6lzvqcLkfiHZNLxf/XNPXsCz2r8GFZ5SMnz5EBn5udMmZ4vtXnw/N+bQTYLfGAVOACSGEkBgLcBQbYOnlVwnwn+v6MiIQG0Y9JHq1uU50vKOe/Hf35A62y743u7Po2+4GuWyP+xuJFUPaxOpc4L7993//97zCTRHm+8a/+PzFbuL9eU8Hug+bxz4iJj9zh5jfq2XOfXlrxpNi+rN3yXh92uOhfO+o/1MowIQQQghxVYDDPv5XE+C1vRmMSEUhAuyGEBsEeE0vBiNy4bYAmyWYAkwIIYRQgMMvwKufZzAiFW4IcKEyrAnw6ucYjMhFsQKcKwtMASaEEEIowKFtgKUE+I9VPRiMSIXbApyvCPN9w4jye4cCTAghhJDQCLCf4381AV75LIMRqcB9+x//8R95hZsiLN83K7oxGJGLfAS4kHHAFGBCCCGEAhx+AV7+DIMRqShEgAuR4awCzOvAiOh7R/9/i9uNsCjAhBBCSEwFOMjxv64L8KtdGIxIRbECnK8MWwowrwMjou8dCjAhhBBCIiXAbo3/VQL8+ytPMRiRCjcF2KkImwX492WdGYzIhRcCrJdgCjAhhBBCAQ6/AC99ksGIVOC+/c///E/H4ZYIGwSY14ER0fdOsQKcLQtMASaEEEIowL50gM4lv1kF+OXHGYxIRb4CXIgQZxNhvm8YUX7vUIAJIYQQEgoB9rsBlibASzowGJGKYgXYqQhnFeAljzEYkYt8BTjfccAUYEIIIYQCHHoB/u3F9gxGpMItAS5UhPm+YUT5vWP+P8bNRlgUYEIIIYQCHNoO0AcOHEh/kF/8MIMRqXBbgJ2IcIYA8zowIvreoQATQgghpCgBjmoDLE2AF7VjMCIVuG//67/+K2e4LcIGAV70IIMRuaAAE0IIISTRAnzcMWXVBxYGIzJRuUJZRwJcjBBnk+DKlSryOjAiGcdWrEgBJoQQQki0Bdgsv/kIsD7ef/99Q3zwwQcZ8eGHH2bERx99ZIiPP/44Iz755BNDfPrpp4b47LPPLOPzzz83xBdffJERX375pSG++uorQ3z99deW8c033xji22+/NcR3332XEd9//31G/PDDD4b48ccfZeA+UT//9NNPGfHzzz8b4pdffsmIX3/9NSP++c9/GuK3336zjN9//90Qf/zxh2X8+eefGfGvf/0rI/Tz4OrDqkmUnTw6lc9C5NYLEc63OZbdObI6n1bn3e4ama+l3TU33xvme8fqHkOY70XzvaruY3OY733ze8PqPYQwv9f070O8b+zes+b3tvm9b/X3wfw3xO5vjflvkvlvltXfNfPfPvPfRqu/oQjz31rz32Krv9fmv+nmv/l2X46a/w+x+n/GzU7QFGBCCCGEAmwpv14KsNMGWG4IsBP5LVSAnchv0AJsJ7+5BNiJ/FoJsFlw7ATYLEz5CHCx8msnwGEQXzdFOB8BdirBbguwUwk2349OJTiXANtJcFIE2E6CKcCEEEIISZwAh6EDdJIFuBD5DaMA+5H9zSZ1Qcjvf//3fzsON0XYCwlOahY4lwDjX7cE2KkEFyLAhWaBCxHgQiWYAkwIIYQQCrDPAmz+IOmmAJs/AIdRgKNU/uxF6bMb4puP9BYjw15LcBBZ4KiVQVOA/RdgN8cBU4AJIYQQCnAoBbgQ+XVLgKM6/tctAbYqN41a9tcP+XVDegsRYTck2IsssFVZu5dl0FZf3FCAKcAUYEIIIYQC7NscwG53gKYAezf+Nx8BDkv5czHZXzfl1yvxzUeEmQUObhxwNgGOSyOsXALs5jhgCjAhhBBCKMAJF2Cz/IZRgAstf3ba/Cro7G+x8vs///M/OaNYEfZTgjkOOFkC7GcjLAowIYQQQlwV4CjPARzmKZCi3gDLbQEutPtzGLO/hcivE+EtRoa9kmC3s8CFlkFbfaFCAeZUSBRgQgghhFCA2QHaVQGOavlzMdnffOXXS/HNR4bdkOCoZ4HDMA5YCXBUxgFTgCnAhBBCCAXYRwHmFEgU4CDLn/PJ/oZBft2WYDezwIU2w4rCOOA4C7CfjbDCLsAICjAhhBBCAaYA+zQFUpgaYIVBgMOU/c1Hfr0UXyci7FcWOOgyaAowBdiLuYApwIQQQggFONECnNQO0MUIcFTKn73I/uYrsP/7v/+bEX5LsJdZ4GLKoKM6DrhYAS5mHHCSp0KiABNCCCHENQEO2xzAFGDvBFj3AVCLI488Uh5DGAS40PJnP7K/hQpvrghCguM8DthLAda/bw4//HD5d8fPRlgUYAowIYQQQmIowLnkN84CnEt+3cgAN2zYUBx33HHyg+AhhxwiWrRoEZoGWH5nf92S30LEN18RdiLBbmeB3SqDjosAlytXThPgxo0bJ3ouYAowIYQQQijAnAM4Eg2wpk6dKk466ST5QbBMmTJi7dq1sRv/62b21w/5dSLBUcoCezkO2HyfejkO2Pz+w5dHhx56qIwVK1ZQgCnAhBBCCKEAx1+Ao94BGuupWLGi/CB42mmnRb4Blpvlz/lmf3MJ7f/93/9lhNcSHKUy6Kg1wurSpYs4+uijRZX/z955QFttZOma1d0z0z3u1+62sbHBZBswGDBgwOSMyQaTTc4555xzzjnneMnpkqPBOWDAOds4u8PMvJm3pt7dhUtdR1dZJR2dc/5/rX8dSSWV8r36tKu2smdXnggLAGwNwKoyQQOAIQiCIAgADABOwU8g9e/fn/3mN79hs2fPVpoAyw8Aq+r/G2/4NYJeM6uAYKdRYFXNoFMZgDdv3swBavTo0SkHwEFmggYAQxAEQRAEAAYAK0uAZQTAdC1Q82eR/CqKCbDCbv7sF37dgK8fCA47Chzk94DDzAStAoDpbxgljaO/L8kOwGF+CgkADEEQBEFQ5AHYCfwmCwDnz58/U+ZkOLouWLCgkuRXYcCvHQh7jQJHoRl0gQIFcD0mkJ944gkAMARBEARBAOBEAWAv8OsUgOl4/u///i+cIKbz5RaAnTZ/dgu/RtvnBoJVR4HDBGA6DypeDETNdDyScb/ofAGAIQiCIAgCAAOA+QOTyky/cLCWAVh182en8OsE1OMRBbYDYJX9gOk8OPnGMRwNA4AhCIIgCAIAA4A1ADZqMgtH00EBsNPor5totcoocDz7AZsBMK7HxLpvAMAQBEEQBAGAAcCZmtTC0bYdAHvt/6safs0g2GkUOBEA2CjCDEfTAGAIgiAIggDAAGANgM36RsLRs1sAVtX82QhujeQFgoP8HFKQAKxfLixTdmb62xbmOml99DckXvvs1wBgCIIgCIIAwABgDYCNPv3ixStXrszkU6dO8bJly5bxz5/QcLNmzbRlzp49q2z9Zk5LS+PbYlZO27V27Vpl69i5cye/HkXddE2p2peoALCVVABwUP2AVQKw/lNIKpwrV65M3rJlC//bVbRoUf6ZoaNHj7I2bdpoy3Tu3DmQbZFN61uxYkWm6U2bNs20vTRNlNN2N2jQgGXPnp27RIkSbP/+/Zn2OejtjxIAm0EwABiCIAiCIABwSABs9n1bt6a66CFZ9okTJ3hZ6dKl2blz5/gwPfCKZeThoEzbMXLkSNPyiRMn8u+fpqenK1kHAcnq1asdrdvLMVYNwG6bPztRKgCw/ju/KlyxYkXD6fQt3+XLl/Phw4cPsxdffFEro3soiG2RTesT69dvL/3NNFqG/gbSttG9IKZdvHiRf5Jo27Zt2rSgjqXseAKw0ygwABiCIAiCIABwSAD8448/KrFVXRcuXGBff/01H6aHYvo9dOgQe/jhh/kv7TtN+/bbb/n4unXrtGnk69ev8+2lafRL06i+rVu3cou6hencrlq1iv9S9HnEiBGG20XrowfyxYsX84d8uUys02zcbB00H20nTR84cCCvlyLhqo5x2ADsJvqbSgD8ww8/KHeFChVMy/r27ct/6f5o3bo1/1tAv/Tyhn5feuklXr5x40ZWo0YNbmrZIJYfP348O3nyJOvatStf9u7du3warbNRo0bszJkz2rz0t7JFixa8bPr06bx+usaNtpfmNdreTp06scmTJ2eaTttJ0WAxHtSxlA0AhiAIgiAIAAwA1gD4+++/V2KruuhBmc4HDRMA06/88H7kyBEe5aL5mjdvzoYMGcKefPJJdvr0aW3eJk2a8PJr167xughce/bsyf3000/zY0XzLl26lOXNm5fXUa9ePQ4Cw4cPN9yuHTt28Lpp3bRddJxEGU0/ePCg4Tg1d5bXUaVKFW0dNB9tAzVdffbZZ9lTTz3F+vTpo+wYA4CjAcDfffedctP1bVaWM2dO/nvgwAHWqlUr9sUXX/BheolEv3TPb9++nZUtW5bduHGD3yPPPPMM2717t1a3GKdlBwwYwDp27MjvG2r9QPcT/d2iMhpesmQJL5s7dy6/T2ncaHtpPUbbS/cH/T01KqP7lVqE0HBQx1I2ABiCIAiCIAAwAFgDYIoEqTDVRQ/msilaRWXly5fn54yG6UFeLCMPL1iwgHXp0kUbp2hV5cqV+TDV1bhxY62sQ4cOvGmlGJ8xYwYbNGgQ7yNJ0SU6P6KsVq1abNiwYYbbTGViG/v378/rEWW0Turbqx+ndRB00PUkyurWrautg+ajiDIN06/Zur0eYwBwNACYXpqo9r/927/xe0I29fmlMhqmX+pD27JlS20ZMZ1M1zPdN2KcXvCIeekepBc4ooyu4U8//VQbp/tn9uzZvEUF3WvydlG9dC3rt5fqfPTRR2O2t3fv3rzsgQce4DBttJ+0HO0HDQd1LGUDgCEIgiAIAgADgDUApubDKkx17du3L8Z0PKmMHngpKkXD9JAslpGHKdEOzUcP7MKinIYXLVqkzfv444/z5DpivmrVqvFfihhTHfJ20XJDhw7NtL10fRAEiG2lh/9ChQpp5VQfTdePU+TKah3ytpqt288xjncSLDfwm8wA/NVXXyk3XVdmZXQv0O/evXv5NaafTs6TJw8rVaoUr0e4W7duWt3UPF/MS7Atz0deuHAhN7VskNdNTflputH2ynXKLlKkCDt+/Lg2TlFm+jtJwwTN9DebhoM6lrIBwBAEQRAEAYABwBoA02dVVNiqrnLlyvG+fzRMD+xiujxMfQanTZvG5xOm80xl1B+RHsDFvPRwTSAgz0vnhPoalyxZMmbds2bN4g/0+m2iafXr1+e/wgTAFOUV66R1iPnF+JUrVyzXIW+rgAmVxxhZoKMBwBTdVG26T8zK6F6h3z179vBrTD+dXLNmTR5ZNaubug+IcYJQ+rugn48SVNWpUydmWqVKlXgLDbs6ZU+aNInfX3K9VM+cOXNi9jOoYyk7Cz6DBEEQBEEQABgALACYmvSqsFVd9MB79epVPvzYY49p0wlkKTMsDVOfWcoqS9tG4xQxosQ7NEz9gufPn68tR801RZNnsSz1g6Th/Pnz80gtDdMxLliwYMy8ZFoHRcvoGpCn0/ooIRANU5NosZyoZ9euXYbrKFasmDavvK0UAaZm3SqPcVS+A6wHYSffAFb1HWCjfQ8bgKn5sGpTn3GzMrpv6JeuQbrG5OnU55z+JlFZ4cKF+YsaSrxG1zL1CxZ108sbsRxlJ3/uuef4fDQ/ZWqnF0n0t4LqGDt2LC+j65rulXnz5hlur1ynbKqH+t9TH/nNmzfzbaP7hPoT04sqMR8dS6pbNm2PyuMKAIYgCIIgCAAMANYAmL5Vq8L0IG5W1rBhQx4pomF60BbTKUJKkSj6XBCNU1NLgkt6sKaHZcpMS9O7d++uzUOm40t10oM6fVuUklDRuaIy6tNLsErrobJ+/frxbLfy9lAUjfrq6reTjjetl+qn64eGqQ6qi5poU3RYrIO2U6yjV69e2jrkbaU6aB55n/3YDoBV9QN2A8FWtoPfRAZgun9UmxJOmZXR9UW/ly5d4v3fxXTqI0z3DU2ncbq26VqlZtKUTV2um/6+yXVSVJe+y03zU+RYTKfrlrJOUxnNQ32JaT1G26uvUzb9HRLroH7ylCm9du3aMftJ2663vH8qDACGIAiCIAgADADWHgz10+Nt2ifqV+hkXjrudB6NypzWYWerelStw6mDAmAv3wN2C79BN38OG4D19xfszPS3kxJqhblOADAEQRAEQQBgALAGwPr64Og6TAA2g2A7EDZbxmv0N8j+v34AGNdjYt03AGAIgiAIggDACQLAXiHYKQCbrROOnmUA9toPWBUEu7FT+HUS/Y0KAOvvXTi6BgBDEARBEAQABgBrAKzfJji6dgLAqqPAfiHYrM4oNH/2A8BG9zccTQOAIQiCIAgCAFsAsFMIThYA1m83HF2rAuCwINgv/Eax/68AYLMXY1H0yZMn+ferE2mbVRoADEEQBEEQABgArAGwfl+jbPo+aSJtr2rrAdhpM2g3UWArCHYKwlbLW603qObPQQCw2d+OKJqylE+ZMiWhtlml9QDsBX4BwBAEQRAEAYCTBID1xyTKzp49e0Jtr2o7BWC/UWA7CPZqN/Ab1ebPAoD1f1ui7J49e7LJkycn1DarNAAYgiAIgiAAMABYA2Cj4xZVEwDL47t372ZTp05lBw8ezDQvfUt0zpw57KWXXuLfMaVfmk6/8+fP5xbTEsV+ADjeEOwXfp1Gf4MG4F9++YWfB/3fnyi7R48ebNKkSXz4xo0brHfv3qxUqVKsZs2abPv27dp858+f59/spbKOHTvye2jevHm87MCBA6xevXq8rGvXrryeRNn/qAOwF/gFAEMQBEEQADghADjMTyE5BeCXX345Yfzoo49qw23btmVlypRh3bt3ZwUKFGDDhw/n0wlqy5Urx6pWrcq6dOnCH9jz58/PDh06xM6ePcuefPJJ1rp1a748DdO0RNl/IwAOMgqsAoTt6vYDv0E2f7YDYALARHG3bt3YhAkT+PDzzz/Pr/9Tp05xwM2ZMyfbt28fL3v66adZ3759ednMmTN5GS1LZTS8bNkyDsnUpHrw4MEJs/8AYAiCIAiCAMAAYA2Ar1+/njAmAKbfdevWsaeeekqbTg/ljz32GDt8+DCPCFevXl0ro2l/+MMfeASLor61atXSynbt2sUTBCXK/tP5IqALMwrsFYSd1Ge0LaoBWHXzZwHA9KIlUUwR23HjxrH09HR+D8llQ4cOZe3bt2fbtm3j95Rc1qZNG74sDT/44INs1apVCbXfwgBgCIIgCIJSBoCD/BRSogGwEQTT8bx27VrCmB7e6bdfv378wVwuq1+/Pps1axZvwkkP+3JZyZIl2f79+9mRI0d4HQTI9NC/cOHChNp/MwAOE4JVOQj4DROAr169mjCmlhBjx47lkV66F+SypUuX8ntH/Mpl1KqClqXhuXPnsmeffZY9/PDD7IknnmBr1qxJmP1XAcB28AsAhiAIgiAIABxnAHYSBabjefny5YQxwSv9UvNLasIsl1F/Ruqv2KFDB14ulxUqVIjt2bOHD1OT57Vr17Lx48ezfPny8WUSZf8FALuJAruB4LBAWAX8qoz+uun/KwD40qVLCePOnTuz0aNH86bNjzzyCG8xIcooQRaV0/2RO3fumLLatWvzMppG/ezFdLpnypYtmzD7HwYAm32CCQAMQRAEQVDKALDKRFhBAvDFixcTxvTwTr+U/CpHjhy8eTONb926lY9Tc2ZqHp0rVy527NgxXkaff/ntb3/Ll6E+j9TkU9TXqlUr1qdPn4TZfysAdhsFjhcEBwW/YUV/BQBfuHAhYdypUyc2atQoPty8eXMOths3buRRYbpv0tLSeFndunW5qasANX9+6KGH+LL00ojuKepHTN0GqPVEgwYNEmb/kxGA9f//6H8iABiCIAiCAMCBAnCYmaCDBGCK7iSKixYtqg3TZ13ooZwS99DvkiVLtLJhw4bxB3ua3qRJEz7Pzp072ZkzZ3jz58cff5zXVbhwYR7ZSpT9lwE4aAhWDcJW6zHbNj/w6zT5lVcAPnfuXMJ4wYIFvMmyGB85ciT/pnbLli15hnQx/fTp07yMplPWaMoETQmvqIxeINH0ihUr8un0gilR9h8ADEEQBEFQUgBwkImwogbAQSXCouNJUJjIFmArfPz4cd6cU56WN29e3gdYjBP06udJBPsBYK8Q7AeE7er1Ar9+Mj/7zf4sA3Ci3zdGptYQ8n1BfX7pU2KJvl/xAmD9/xEAMARBEARBAGgXZScAAIAASURBVOAIADBlhk0mU3POPHny8G+fzp49W2vymQz7pgfgMCHYCRS7Wd5sO8Js+uwl+isAmPrTJpunT5/OP3dUvHhxfg/RvZMM+xUVADb76gAAGIIgCIKgyAGwykRYUQNg6jebbKZstxTNateuHRs0aBBvrpkM+2UEwCoh2AsIe3GQ8BtE9NcIgE+cOJG03rRpE9u7d2/S7I8MwFH7BBIAGIIgCIKgSABwkJmg7QA4zG8B0/GkJsNwYpjOF4GcEwD2A8FBgbBX8A0Cfr1GfwUAHz16FE4QA4AhCIIgCAIAxxGAo5QIK2vWrOKhCU4A0/kyAmCvEOwEhP3CsJP6vcBvvKK/5GzZsuF6TCA/+OCDkQVgO/gFAEMQBEEQBABWnAn6448/zuRPPvkkxp9++qmhP/vssxh//vnnMf7iiy8y+csvv4zxV199lclff/11Jn/zzTcxvnv3boy//fZb7goVKrCpU6dq4999912Mv//++xj/8MMPhv7xxx9j/NNPP8X4559/NrQelsygygjAjKKUZkAXNgQ7gWK3ddhtV1Dw6yf6SzY67/rrg6y/hsyuNf01qb9mxbUsW3/9k/X3iNF9pL/X5HuR/r4a3bNk/b2tv/eN/j7o/46Qjf7e6P8m6f9mGf1d0//9I+v/Rhr9HVXR/zdMAHb6DWAAMARBEAQBgCPxLeBkAWAzCPYCwE4g2AsAk+lzL6VKleJ1OAVgMwi2A+AgINgNAKuGYK8g7NVhwa/q6K/ZObeDXzMAtoNfpwDsBH7jDcBGf2tUAbAT+E2WTyABgCEIgiAIAJwS3wIOqh+w2UNp1ADYDoLpXDzwwAPs7NmzMaBgBBTJHAX2C8FBgrDT9QcJv0FEf50AsMrob9gArL+vAcAAYAiCIAiCAMAAYAMA9toM2gsAN2zYkPXv398QFoICYDMINgKnRINglTDsdp1hwm8Q0d+oArBRdwMvAOwEfp0CsB38AoCdwS8AGIIgCIJSFIDxLWD3AJwM/YC3bt3K8ubNy9fnBYCdNoMOIgocVFNoKwj2AsJOoNhPnVbgawa/YTZ9jkf012vzZy/9f6MKwGZdO8Lq/5soGaABwBAEQRCUIgCcLJ9CSrVEWKr6AZNp+7Nnz87S0tJikmFFrRl0kFHgeECwSlttoyr4TfXob9gAHGQCLAAwABiCIAiCAMDIBJ0SibCMILhLly6sVatWmbJBB9UMOhEhOMog7AV8w4bfMJJfRRWAE6H/LwAYAAxBEARBAOAUBeBU6wd89OhR9vDDD/Nj4ReAg4gC+20KHSYEhwnCdtvhNeqrAn6jEv0N+vNHMgDT38Qs0ndz77vvPv43Kujmz8mQACsqn0ACAEMQBEEQANgSgIP8FFKqJcJS1Q/YbTNoqqNQoUJs1apVpt8EjncyrKCiwHYQ7BeEg4BhJ+v0E/V1A7/xjv5GsflztmzZNADu2rUrEmAlWAZoADAEQRAEAYCRCMsCgBO1HzDtP0WnqlSpwps916xZ0/CbwPGIAgeVECueEOwHit3WbbWtXuE3zKbPfqK/UQDg1q1bs3/5l3/hFtHfZO7/CwCGIAiCIAgAHDEARj/gzAC8YcMGdv/99/MHvd/97ne8+fPLL79s2Cw07GRYqqLAYUKwVxBWabvtCwt+3TR9jmr01+vnj8jLly/n91Xt2rUj3f8XCbCM4RcADEEQBEEpBMDoBxwcAHvtBxxUM+hOnTqx3/zmN/xB709/+hM7cOCA4TeBEyEKHCYEOwHhsGFYBfgGCb+pFP2le5H+JlLriuvXr4fS/Dmo/r8AYAiCIAiCkg6AKeIHAE6OfsBuo8A5c+bkAFyxYkW+D0bfBHYLwGFlhFbRHzgMCA4Shp2u28m+xAt+g47+qk5+5RSAyRQFjsr3f6PY/xcADEEQBEEQANghAKMfsH8Apv2m/olTpkwx/ByS02bQiR4F9gvBbkHYDxC7XYeTbc+fP39MxmIYThTnyZMnsP6/AGAIgiAIAgAbAjA9gOBBDIbDdYECBZREfMlU3+LFi7mXLFnCli5dypYtW8ZNEcwVK1Zwr1y5kmcIJ69evZqtWbOGe+3atWzdunVs/fr13NSvfOPGjdybNm1imzdv5t6yZQv31q1b2bZt29j27du5d+zYwXbu3Mm9a9cu7t27d7M9e/Zw7927l3vfvn1s//793GlpadzUZP/gwYOaDx06xA4fPsx95MgRbvq0F/nYsWPcx48f13zixAnukydPcp86dSrG6enpmk+fPq35zJkzjn327FlX88vrkdev3zaxzWIf5P0S+yr2XRwLcWzoOMnHjY6jOKbiGNPxFsdenAs6L+Ic0fmic0cW55LOK51fca7FuafrQFwTdH2Ia4WuG7p+xLVE15W4xuh6E9ceXYd0PdK1SabrlK5XunYBwBAEQRAEhQbAZFp+0aJFmR6g9Q/R+gdp/cO0/oFa/1AtHqz1D9dmD9jiIVt+0DZ72BYP3EYP3eLBW374lh/AxUO40YO4sHjolB/K5Qdz/QO6/iFd/7AuWzwEm1n/0OzWdvUbbZPRtsv7pt9v+ZjIx0o+hvqHdfmBXX5oFw/u8sO7/gFefogXD/LiYV6+VsT1I64no4d6/YO9/HAvP+DrH/L1D/r6h33xwG/00C8e/P2Cr2jyLACY6jcDYPmeNYJfca/K96h8f+rhV9yXAqD8wq/+XrODX6fg6wR6CXD92g0Mm92f+vsvCAjW3z96CJbvGfl+MbtPxLVjBMHyfaG/HwDAEARBEATFFYBl+LUDYPFQIz9Mm8GvPqqkAn7Nok1O4dcIfM3g1wx89fDrFnyDAF0VYOwHhPXHxwiEjSDYDITdQLAMwKohWH7AN4Ng+Z6wA2D5wd8r9Or7+soAHET01+je1L+QMroXVUZ+nUZ9nYCvHl7PnTvn23ZA7BSEzaLBTiBYfz8Z3Uf6+8fo5ZH+flERBZbvCfl+8AvAVvALAIYgCIKgFANgpxCcxaD5pNvor3jwMYooGcGvWdNKowhT0PArP4Q7gV+n4KsKeuUmlF4cFAw7AWE7CLaLBgcBwfK1ZvRgbwfB8rXuJwpM951f8HUKwKqjv06aPhvdg2b3nhn8mjV59gK+TqH3/Pnzju0Fht2AcNAQ7LYptPyySFUUGAAMQRAEQZBnAPYaBc5i0HzSKorktOmzyuhvGPBr1OTZLOrrB3z9wq78AG1kv1CsAoTNosFemkSrgOB4NoU2iwK7BWCzJF8CgPX3bhDRXz9Nn/3Cr5uorxn4OoXdCxcu2NopENuBsJtosF8IdtIf2OqlURBRYAHAdvALAIYgCIIgKBAAdhP9jTf8mjW39AO/bqK+bsDXLfDaQa5XuwViLyDsNRrsBYLt+gOrhGA3TaGdRIGdArAV+BoBcNjRXzdNn1XDrxfwdQO7Fy9ezGQ3UOwHhFVAsJv+wFZNob1Egc3uC/394BSAvfT/BQBDEARBEADYFIBVR3/1ia/8Nn12k/QqKPh1GvW1A1+3wOsm+6xdEh4nQOwGhN1Gg1VCsN+kWE6bQgcVBbYDYCfgKz4BpQdgoz77YUV/7Zo+Bw2/TsDXCei6tRUM24GwKgjW31N+m0KrigIDgCEIgiAIiiQAu4n+Pp4Pn04K0vkzjq9f8HULxUGAsF00OGgINusPrI8CG0W3go4CGwGwE+iVwVcGYCfRXwHAbqK/ThNfyS+inMCvAGCrhFcy/OpfIllFfY3A1wnwXrp0ybXtYNgOhPX3pUoIVtUU2k0UWL439PeF0SeR/ACwHfwCgCEIgiAoBQHYCQRbAbBZ9JeW+X/f3oYDMh1fP59oyZMnN14kBOxHHnnEVxSY6nAKvGbg6xaA/UR/zVpk+Gn6rAp+vYKvEdBevnzZsZ0AsRMQNmsSrX/55AaCjV4oWTWFjlcUmK5dPfyqAmD6nwgAhiAIgqAkB2AvUWDx8Oym7y8H4G9uwgGZjq8R2Dr9JAs/P9/dgQM0HWM/UWA3AGwGvkYAbJf8St9VIR7RX7Omz0HBrxX4GoHtlStXbG0HxHYgbBUNNoJgu0iwm/7AcksKVVFgs24CdsmwjABYVfNnADAEQRAEAYBdAbBV318OWF+/BQdkOr5+vknKz8/dd+EAbQXATqLAdgBsB73Cf/3rX3ldQUZ/5b6/TqO/bpo+G/X79QK/VuBrBr1OYNcLEJvBsBkIu4Vgo+4GfppCq4oCu/0kEgAYgiAIgqC4AbCbzM+0zP98+TockM0A2On3Se9F6N+GA7QAYK9RYCMAdgq9AnyFVQCw18zPfps+m8GvPuGVV/h1A75Xr151bTsYDguCnTSFDiIK7LcZtFcA1v9fAwBDEARBEADYFID1EJzFpP+gVeZnDsBfvAIHZDq+TmHXDID/56s34ABNx9joQd9pFJiWdwO8RuCrB2Cj5s92ya/MsrPbZX72Ev2Vmz7nypkTfckDdM6M4+sEgr1GgfWZ0+2SYZk1g1YFwEbwCwCGIAiCoBQBYLdR4Cy66JGT7/5ywPrsBhyQZQC2+u6ome+9oHgVDtAyAHuJArsFYCPwNQPgIJNfeYn+GjV9zsL7qb8HB2Q6vlZNofUJseIVBdYDsB5+AcAQBEEQBIUGwFbf/aVl/vvTa3BApuNrBLZOv0nKAfjzl+EALQDYTRTYLQBbQa9KALZLfuX0u7920V+56TMH4G9vwQFZALCKKLDZJ8ScJMPyC8B++v8CgCEIgiAIAOwIgK2aP4uHGQ7AH1+GAzIdX6ewawbA//3pS3CA1gOw0cO+VTNoMwB2Cr16AFbR/Nlt9FfO/Owm+qsB8DfvwAGZjq9dQiy7vsAi4ZmXKLD+5ZBZM2gAMARBEARBoQCwDMFy9EgfQTJ6kNYA+MMLcEA2A2Cj744a+d4LiitwgKZjbJX1Vh/xMur76AV29f7ll19iADiezZ+dRn8paVMWZJIPPJO8m4RYbqPAqppBewFgO/gFAEMQBEFQCgKwmyhwFpMEOmbRX3qgoWX+7wdnA3f6tvmaXzu2jv3w1tGY8huHV9vWcfe1Q47mi5Lp+DqFXVMA/ugiHKCNANhNFNgvABP4CvsBYCfJr6w+faT/7q+T6K8AYGR8DzaTvNVnkawyQotzbJYMS2UzaBmAVUd/AcAQBEEQBAA2BWCnya9iAPi904H7j//+B9b+hee4G9WswHLneIQtmtCPl711YgP73W9/y768vs+yjvQt81jlssVD2V5VlgFY/31RJ+YA/MG5yHjd7JGZfG7nYl52YM0M9uNbx/hwiwbVI7XdVhYAbJT11kkyLK8ALIOvGQDrX1wF0fzZT/RXA+DPX4EDMh1ft59FUhEFdpINGgAMQRAEQVCkANgq+hsDwHdOBu7cObLFjH92ZSf785/+yH54/QAfv/vyvkzLfHRhG/vHu8e08fQtc1jlMsUs1/PLW4f5ckZltI4vX9odyv4K0/E1Alujb44aOawXFG6Afly/9jHeu3wyLyuYLye7un85H6YXHIn0kkIGYH20yyoK7BaAjaBXD8BG/X/j2fzZKvpL367lAPzZdTggxwuAraLA+pdCXgDY6fd/AcAQBEEQBAA27QecxSCBjlXzZw2Abx0L3ByApfEfXt3HsmX9C/+Vy9M3zWS1K5XiJtjNnys7e+3QCq2MA7DJOhaN683np3mKPJGbvXV0lVbWq01DDmhlixdi3VvVy7Q9QZmOr1PYNQXgEIHdCdCbldFLDfHCQv/C47Ujq/gLDPHCI2r7JB7yraLAZs2g7QDYDnqNADiI7M/65s92ya/MvvsrR38FAP/3J1fhgEzHl461gGB9Rmi7ZFhemkF76QcsAFj1548AwBAEQRCUggDsNAqcxaD/oFXzZ3qgoWX+690jgfuP//571q5JTc0EqDsWjtLKCZjo99TGGRyMP724hY9vmTeCtahXWSurXKaoYf1nt85mZTLg9ufX9/PxK7sXZADvY3x4w+yhrF7VMuzvbx/U6gxrv40A+OrVq44d1gsKN0BvVkYvHt47vSHmhcY/3jmccf6q8Bca/do3vhcl3rMocvukf9B3kwxLBmA3sCv7559/5nYDwKKJqlX/3yCbPxP8agCMjO+BZpKXAVhlFFj0EZcBWN8M2i8Aq2j+DACGIAiCIACwYwC2iv5qAPzOwcBNUHtqwzTuw6smslE9WrBSRR5n39/Yxctz53iY/1L5C7XLa8vdObmGVS79lFYmhvUe1rUpWzq+d8y08iULs+t7F7J2jWvwdeq3J4z9puPrBniNADgMUHcD9GP7vBhjejFBZfRy4k76upgXGvSyoVOz57TlL+6Yy2pXLBW5fTIDYCdRYFreD/TKlgE4Cv1/9c2fCcJkAD5x4gQyyYeQSV4AsBwFtkuGpboZtF0/YAAwBEEQBEGeAPjGjRu+ANhp8qsYAH57f+DOnf3hTNNa1K3ItswZElN+av1k1u756to8d06uvAe9v5aJYb37tWvA1k7tFzOtdsUS7MrOOax1gyrs+NqJMWU5sj0Yyn7LAHzt2jXXDusFhRugXzttQIzpJQOV0bmhFxbyC41uLerwFxpje7fWLMqitE9ywh+3ybDcArAR+JoBsNH3f932/xVNXq2yP9s1fzaL/goADiOj+trZI2JMGeVF2T/upPNp9GtVB2Wf37l0YsJlkqdjbRUF9tIMWvX3gN0AsNv+vwBgCIIgCAIAG0JwFl3zSbvmzwKA//PNPYE7d/aHYsb/9tpOVqZYAbZ/6aiY8pPrJmYAcFVtvtvHl7PKzxTRysSw3ltmD2JNaj2rjX9yZg2H3J9ubGcLRnWJqZPqCWu/aT16qH3ppZccO6wXFG6A3qzsHgCvjHmh0at1PTaqR3P+8kL44raZkdsnIwB22gzaDoCtgNcJAHvp/2v3+SNVzZ81AH4vPXBT0jw5+VrtyqVZvWrP8rIvr+/lzevp16qO985tvdc8P4TtVWUnAGwVBRZ9vWUA1jeDVtEPmLYzqOgvABiCIAiCUhCAnUSBsxj0H7Rq/qwB8Os7A/fv//VfMuC1sOb8OR9hnZpU18o5AGf8nlwznrVrVEWbfvvoEj6/KLv//9wXUw/50LJRvLz5c+VZ7fLF2dBOz7OCebKzHXMH8el/e2Ubq162KJ+X5mlYrXQGHD8Qyn7T8XUDvEYAHAaouwF6szJ6OUEvLOQXGrsXDmcNq5fR5vng1Er+AiJq+2T0oO+0GbQegN0ArxEAq/r+bxjNn0+e/DUx2u0TgZuDq8G0j85v5cPi9x83j7LPLu9gd2/sZTsXj2VvHVujzf/emU2G9ch+7fBKvpyoT3b65tnswMrJ7Jc3D/F10LqC3m89AKtuBm3UD9jL94ABwBAEQRAERQKArZo/0wMNB5rXtgfu20cWxfibC2tjyj84vpT//vTSJvbF2VXa9L+9vIV9kr5cK9PXQ/7u8npt/us7ZrAdcwZq9cl+ZfcsXk7DHIBD2G8jAL5+/bpjh/WCwg3Qm5XRCwZ6YSG/0CDTC4liBfPwlw/0e2HzlMjtk/67p26aQdPyfqDXCwAbJcDy0v9XNJWVmz/LAGwX/RUA/F+3jgZu3q9cGv/w3CaW45Gs7PtX98aU3zm9ng9T4rWxfdvwzPBb5o+IKTNbR7dW9Vj1ck+zUb1a8UR9K6cO4NP//s4h3sf9hecqsmHdmvO6ixfKx+sLer8FAOujwE6yQQf1PWA/AOyl+TMAGIIgCIKSHIC99gN2CsByIh1a5j9e2ZLUfmXXDFa51JP898er69nycV1ZkxplQlk3HV8noCvOuyEAv7Yt4f3NhTXs9pGF7G8vb47ctgkAtooCBwnAP/30k2YzANa/uPKaAEt1/18NgG8eDty//7d/ZZVLF+UuU7wgzyy/Z8lYrZz3Lc/4vXNqHcuRLSv7+bV9fPzVA0v5/KJMzKf38XVTOfyK8W9e2snn/f7lPWzl5P6sdcNqWhnVQ9tDv0HvtxkAWzWDNusHHGQiLK8AbNS9BwAMQRAEQQBgRxCsf3i2a/6sAfDLG5PeR5cPZ3UrPZ0BwoVY16bV2LcXVoayXhmAxbl143svKDbDAdoJAFs1g/YCwDL0GgGwqgzQRgmwjL7/67b/L8FvDAC/cyBwU5/+OydXaz68agLLn+tRdvPYCl5+L7naAV52L1v8P5eVy8Sw3hP7t2ELx3SPmdaibiV2asNU1qlpLZa2fGxMGUE11Rf0fnsBYCf9gI2+B+w1EZYRAKts/gwAhiAIgiAAsG8AFg/5HLCur4MDsnxO7c6t2UPff7y8AQ7QegB20gzaCwCbQa8ZAHvJAC2SGflNgGXU/5fgS9//99SpU/cA+K19gZsnVtNNG9W9GZs5pENM+Z0TKzIAuIjhslRmVA956sC2bO6IzjHTXqhVjl3cOoP1al2XbZgxIKasYN4cvL6g95uOLx1vq37Abr8HrCoRlnwvBAnA4mUgHhEgCIIgKIkB2Esz6CwGzSetmj9rAPzSajggZ7H4trNjAMaLhMBfUogHfS/NoM0A2Anw2gGwqgzQXhJgiT6mVs2fBQD/55u7A/e9xGr/HP/kzGrep/zY6nEx5bePL/s1W3zmZalMX4/w9d2zeX3fXdvMx984sIDP+9ONbezkugmsZJH8WtmW2QPZ7377W15f0PstA7CTfsAqE2E5AWBxLzgBYK/NnwHAEARBEJTCAGwFwVkMmk86AeB/XF3hynfPLGTn1gx3vZzsa5vGsBNLB/Hhd/dN5XWKsh/OL9aGX98xgX11ar6vdcXTRgBs9NBnZn5+rq2GAzQdY6uHfSsAFg/+XmBX9o8//sjtBYBVZYD2kgBLA+AQkpURcBKQkimLfJHHc7I5wzpkyiQvZ443KpPrERb1LBnTlWeQp+Xp98z6iVodVEbJ82j6wPYNYpK+BZ2kzQyAnXwP2EsiLC+ZoGUAVt38GQAMQRAEQQBgxwBs1fyZHmo4YF1Z5srzB7fgnzV6L22a62WFV49tx8Z0qc+HyxTJy2YPaMaHaZqYTm5b79kMUB7oeT3xtgBgK8i1eij08oKCvGlyF/bZsTmewf2jw7NY35bV+fBbuyez9BVD+PAvl5bxcTGfXJbILyn0AOymGbRXABbQK9srAKvKAO02AZYGwEmQqE2YErVRwjZ52neX17Evzq6MmUbgTNPDSNLmFICNEmF5BWB9JmgAMARBEARBgQOw22bQWUz6D9oC8KXFrlzsiRxsQOvqbEznuq6XfXfPRP67enRbbfm7p+awX84v5MM0Ta63bd2y7MTifpZ1fnRwKq/DbH2ibjfL0XQqd7t/etPxNQNdJ+bn5/JSV35582j+gmJKr+ddLyv87p5JrFKJJ/jw8pFtWPOapTJNJ68e046/pPC6nijYKwDLD/5+oNctAHv9BJKqDNByAiwBwMmeKO3jk0tY/pzZ2JLRndiJVaNYzxa1WLuGlUJL0iaOebwyQasAYD/NnwHAEARBEJTiAGwGwXYArG/+LAD47xcWOPbZFQNZ3fJF2KeHprJ8ObKyn8/O08ra1i3Dji/qYzievrQfn7/S04+zckXzsQGtqrHRnerEzEfT7v/jH7hpHqM6ZX+YNpnXV6ZwblY476OsxwuVtO15bcsoViDXw7weKh/ftR6vi8q+PDqdVStVgJUsmJN7RPvavB4qo+W7Pl+e10fL0XSaX2wLbSOV3dk7wdHxouOrf/Azi44YmQPwxUWu3DPjOMwf1Izlzzje8vS7J2ezr47N1MZ/OD2XfXZ4esx4+tIB7OVNI9m7uydkgO7jMfOJ8jJF8vDyX84tYKtHt+EvKay259q64Xw5Wl6eTsufWzGIl9P4e/smxZRfWj2El9N8Hx2Ywn9FGc17YlG/mGXE/tF65P20swBgr82gzQDYDnb9ALDKTyDZZYAWSZf0CbBiADgFkqV9fGIhmzGgJc8iv3xsJ/bXl9aFlqSNjrUXANZnghbXQBDfAhYAHET0FwAMQRAEQQBgtQB8fq5jt61Tmu2d2YUPN6v+tDYsyo4v7JVp/Oczs1nuRx5gF1cN4NPvHp+WAZGPsNEda2dajqaJ6UZ1ym5Y8Sm2amSrmHkXDW7Kh4s9nl1b7vtTMzOA9wleTuMd65dlk7vX15br2qhcBujm58PrxryozUeeP6AJG9CyqlZ/3XKFeX1Oj5cAYDfQqwfgv1+Y79jfp8/mLxrot3HV4uzAnO5a2apRrdnoTs9p48cX9c6A19J8+OausXw5Gm9WowT3vZcC/5zv6roh/KUAvaCgsg/TJvE6RR1G20IvGmqVKaS9VLi4ehAv+/TQlIxzlINvIy1PpmuEyn4+O5evn5bt07yy9rKCtpHK6WUGLUtl9Lttckdt/xpWKsqn0bDTY6YCgL3Art4//PCDIQCr/Aawl08gWWWA1gAYydQCTdJmBMBuM0HHE4CNWrcAgCEIgiAIMgRgN82gaXk3/X81AD47y5G/PDSR5XjofvbWluHs5vaRbNWIFhlA+KRW3rbOM+z4/B6Zxq+uHpABTPli6prfvzEb3aFWpuVomphuVKds2hZ5nNZTq0xB9uHesRyA5bJN417kddFw7kf+wr4/MU0ro/0R29ewYhE2oEVlvm/k6T3ra2W0/IFZXRwfLzIdX/3DoJwp1c78/Jyb49irRrbMAPYqfPjA7K78JYFcxl8u/Dp+fEHPe8ckY5heZvCXB7+Wze7b6N5LAd18N3eM1qaLOkWZ3tN7NWB9mlXSxi+uHJABso/x4R5NKvByUbZpfFt+Xmh43ZjWrHGVYlrZjfVD2O9++xu+7re2juDn9ufTs3jZl4cna8vRthDE3z021dUxo2NslPXWaT9gPwBM0CtbJQCbfQNY5SeQBAAjmVqwSdr0AKzPBO3nW8BOAFj/LWCz+8Br9Neu+TMAGIIgCIKSGICvX7/uOQrsFYD/dmaGI8/r34gVyPUQq1Q8n+YH77+P3dw2nJe3ea4UOza/mza/GD+7pBerVbpATF3LhjbNAN2amZajaWK6UZ2ysz3wf2LGr68dyKqVfJzd3jmSg5ZctnNyO14XDefKAKafTk3Tyt7cPJTvCw3XebYQG9y6Kls5ornmtJmdbLfFzOKh0Axw33jjDUPHAPDZmY5dpnAutmhQE3Z8fnduOj93do3iZasy9oWOrZiXytvWKcWH770UmKqVfXlowq/gHzvfze0jtOmiTlGmN70coWXlafTSguomiKW6xPSf06ffA9mM4a4Ny7KdU9pn2i+an9ZXrmge7TohU52irE/Tiq6OFzlMANYDbyIBsFECLA2AXSYeo8zu6yd08pW87OCCfmz12A58+OjiAVpyNqpbTtRGSdrk8URM0uYEgO0+hWQGwPpvAccDgO2ivwBgCIIgCAIAWwKw0+bP9FDDATh9qiMXyPkQu719eMy0yV1rs8GtKvPhLg3KsJXDmmpl5Z7KzY7N7cK+O0qR4z+xD3aN1MrqPFuQjW5fnQ+3qV2Sz0fD03rUYcPbVNXmk8v0Lv1kTnZhWS9tfHynmtqytK3X1/SLWR/VRcONKxeJ2U7ajkrF82r7Q/WIsk/2jmavrh9guy1mzqL7NqYZ8JqZn5/T0x35yirqZ/1gBqiX1FzuqQxYbF+Dl68c3kwbJh+e04XPQ8O03DeHJ2plH+wede+lQMbwsXldtflubh2mTRd1ijK9G1QozNchTyNYpfXQuXt1wyBtOk2jFxM03KNxObZ2VIuY5QrnycbXTdMbV36Kb5Ps745NzrR/Ti0DsJdm0FYAbAe8QQMwQY4XAHb6DWANgF0mHpvRtwlP1HZiyQDPycsoAduYzvX4cNfGFdniYa0yTSdTkjaalshJ2mQANvoWcDwAWP4WsBUAq2j+DACGIAiCoBQDYKfNoD0D8KlJtj42pxOrVfrxTNM/2DmM5cj6J/bT8QnsyopeLFe2P2cAcSXWoHwhli/7A3w5mm/lsCZ8fPiLVVidsgV4+eh21XhZm9oltPmur+7D6xjQvIJWViz/IxnQlUdzs6pFedmFpT0yQDcrn5fmK1kgO/vm4Bhedmp+F15PqxrFWbWS+fj6aB4qu7llUAZQPcwaVyrMt4WmU71URstTPR3qluLbSusW2yZvp1PT8bUC3DfffNPQMQCcPsWRuzQonXGcX4iZdnv7MH4cfjqRcQ7nduYvJUTZgOYV7x2TjGHa31m962ll/3wpMIUvJ+aj+orlf1Sbj9YnyvReNKARP/5iPG1Gh4xzkZ8P00sG2l55fbSdYj4C5G8OjdPGeRPorUP4+unlhiij341jWmjbcu+lyhRXVgHAbkHXLQDL928kAfjSIlcunO9RtnxEa9a8RsmY6Z8dmc5+ODNXG//q+Ex299Rsbfy9/ZPYtqmd2bt7JvAEbPcyxv9zPvqd0acx69uiKp+XyihJG81rti20vn2zu7OjC/uwX84viCn76OAUvr7Xt47m89H26ZdLXzZA2zZ5WZq+a0bXmGXE/un308qqAFifCTpeAOyl+TMAGIIgCIKSHIC9RoE9A/CJCbb+Yu8IbqOy21sHsZ+OjuPDH+wYwraObcFubhrAPtk9jH13aIw2H03bOaEV//0mbZRWn34+Gj+7sIs2TPPLpvWJeamevZNfZKfmddK2QS6j6VTHskGN2IBm5bUympfWQfW9uqYPq1YiX0wZLUfbSsvK2yVvpxPrAdgMeM3Mz8/Jibb+5sBo/iLiu8NjM5UR5O+d8iIfblb1qQy4fIzVeuZx/gKgTe2n+fRPdg/n4E/HgTyte21WqVgeXnZsTkdtPjK9TKAXAx/sHMpWDm3M7vvDv3J4lX12YVf207HxGWBdktdL20AvHV5d25fXQWVUJ70UofJ5ferdA+Bf1zG6XVU+Xu6pXKxL/Wf4783NA3kZnUt68dGqRjH+S9tA0+mXlnNyvGQDgP0DsJtM8pTVnZKfUcZ16rNNGeXNMslTpvhVo17kwzundeGZ3Ye0qcGTnVEiNpFJXsxH81Dmd0qERhnbRZ2iDr1Ftvg+zauwjg2e5cuJ7aHM9ZSYjdZHSdnuJWy7l0n+5q5xfDnKPE+m6feSuC1g36fP4dvWqvYzfFma7+q6odq2UF20jbRup5nkkxmAnUR/AcAQBEEQBAC2BWAn/X8FAP/1+Nikc+/GZdikztXZ1/uHs3c29mWlC+VgJ+d24GUrhzRibWoVZ+9vH8g+3zuUdahTgs3tXSeQ7RAArAfbt956y5HvnZ9xSn1ry4CM/R7musyrP941JOMc9DMs+/bgKG0eAmSzcoLhr/eP0KbTMNUpT/NqAcBe+wGrAODvv/+eO2EB2EUmeUq4dmBONy3r+/ReDU2zvlO5yPROgHlz5+iYekTGeHk++tVnkpezxcumDOVyJvvZfZ/Xsr4TuN7YMFQro8RtIkN8q1ql2LJhLbSye0ncHuDDlEyO5hVltD88qduv20Lb7eZ4JTIAq2r+DACGIAiCoBQEYCfNoD0D8NFRSeev9w5hg5uXY5WK5mI1S+VjG0Y0jimf26s2q/Z0Hl4+rUt19uOhEYFsBx1fK+h9++23DR0DwMdGJ50PTX+RFc79ENszsQU7OrMNa1CuAJvUqRove3/bAF62dlgjXja8dQXWrErhwLYlHgAsgFfvLAnaB9hpVvRP08bzxGyU2Z1MmdYpsZ5Z1nfKCE/zUNZ5SoQm17VzSgctY7yYj4bpV59JXpTpTa0X5HFaDyVoo+28l5Ttn2W0XSKTPPWZv3t0ilZGWeXF/DTPvc+uPcPdqlaJmLK90zu5ziSfqH2AVcEv/U8EAEMQBEFQkgOwlyiwGwAWDzUcsA4PhwMyHV8nwGtmfn6OjEhKX1jQgXVvUJJ1eK44Wzu0YUzZO+t6ssHNn2VtahZls7rXYN/uHxLYdtAxlh/4rZpBewVgM+ANGoDDygLtNOHYtB71eBI1OVEb9ekWydJoXE6cNrhVFZ7c7PaOEaxkgRwxdW0c20pLeka/NJ9RsjeqU5TpTTD+08mp2jglZqP+55RUjWeZl+bdO62DlvCN+sHTNslJ40QStw71SrPxnWrFJGmjTPhiW2jcbZK2RMwCrTL6CwCGIAiCIACwKQC76f8rAPiXQ0PggCwA2Axw33nnHUPLAPzLoaFwgHYDwEYP/zIAOwXdRARgq+8AO004di+T/LCYaVvHt+KZ2UUyNErORsOUuI1AUyR2o2UpIZsooyRtIukZ/Yr5KClaj+fLavVTkjZ9cjhhSv42r1+DmERy97LAT2G1Sj/Blg1urCVbo/WJhG+0PjmJG61PJHGT94d8fU1ftnPSi9q2iH1wk6QtEb8DrCr5FQAYgiAIglIIgN1CsGcAPjAIDsh0fJ1Cr5H5+Tk4GA7QKgDYL/h+99133GEAsIBgMwCWm0E7BuCTk2ydNq0dT4amn/7TsQk8gRtllP9k9wieFI0StZFp/pVDm/D5Xl3bjyc9o4RoVDa+Y417meQzyuhXzPfFvlG8nJK10ThB54P3/3umRG1URuujpG80P/VBp3lpe6iMtofKaF6xPg7Av25zl/qleRktt3ZEU61O8uCWlfh0SiRHCeNo28W28EzyDo6XsBEA0znxAsB0DXgBYLk7gCoAdhP9BQBDEARBEABYLQCn9YcDsgzAMtjevHnTke+dnwFwgI4XAAvolW0EwPJ97AaA5X7AAoDlKLAbAJabQQsAFhBM2/zXE+OV+tbWgezHo2MNy97Z1N+0zKs/3j2Ufb53eMy093cMZt8eGq2NbxnbnHVvVFqb/+u0kTHbVLrQYzHLUznth99tMwNgOjdWAEznNl4ArLr5MwAYgiAIglIYgK2SYQkAdpoASwDwz/v6wAGZjq8V+L777ruGjgHg/X3hAC0DsJdEWE4A2Ah2/QCwuIdVALCAYDMA1vcDJhCTo8AcgI+NSTpP6lSd1SnzBEub0pptGdOUJ2Z7aUV3XnZoehs+TtOpvFqJvGzl4IaBbAcdX3HM3QIwnWM9ANO14AWA9feAGwD22/wZAAxBEARBKQTAbqLAngF4b084IMsAbAe9Rr53fnrBAVo1ADuFXa8ALEeB3QCwvh+wgCO3AKxvBp0lSTPJk9OmtGS9ny/NM8q/tLxrTNnlJZ3Z8FYVeDnNF9Q2CAA2ygAdFADr7wU3ABxE9BcADEEQBEFJDMAvvfRS+AC8uxsckOn4WkV8b926ZegYAN7dHQ7QKgDYD/QKf/vtt54BWECwGwDWN4MmcNJ/CskxACPje6CZ5J0CsDiHdt8AdgLAcncAPwCsIvoLAIYgCIKgFAdgMwj2AsB/uf9P4sECDsAPPfiAK/DVm5bHcQzW999/f1wAmIBXby8ALEeBBQDrvwXsBIC9fgs4CzLJB55J3uobwHoANkqAZQfA4prRA7DV58AE/LoFYC/RXwAwBEEQBCU5AHuNArsFYP3Ds3iANooiGT1IGz1MGz1QGzWtNIouGT1gy9lm09PT2enTp7nPnDnDffbsWXbu3Dnu8+fPc1+4cIFdvHiR+9KlS9yXL19mV65c4b569Sr3tWvXMh1v+diKBzZ6iBN9215//XXuN954g/vNN9/k1n/qSG72bAa+t2/ftrRRRFj0D9Z/LolM2yC2R2yf2F65b57+QVR+2BTHg46NOE7iuNExFMdTHF861uK4i/NA50ScH3G+6NyJ8yjOq/6h3iiyZdS80+gh3+hB3+hh3yjiZdTv0c23gJ0CsBHw2gGwnAnaLJGdHoD1maC9ArDTTNAcgA8MhAOyDMBeM0DrAZiuCT0Ay/eCGwCma1UAsNfoLwAYgiAIggDAnqLARgBslQE6agAsw6/+UysEUAKAZfjVAzABmR6ACdz0AEyApwdg/XEV8GsHwASeMgCb9fk1A987d+7E2AyE9Qmy9BAstsMOgMWDqf7BUw/AAoJlABYQLAOwgGAZgPUQLM6fDMFyYh+zh/soArC+CagegJ2ArhMA1n8KSb6X4/EpJLNEWByA0/rBAdkIgO36/zr9BrAegJ18A9gIgOmadQLATuEXAAxBEARBAGBHUeCoArD8QG3UtzDRANgo+qsHYCfwqwdfKxC2gmAjANZDcCICsPyAb/eQH28A9gO9RgCs6lvAQWWClqPAWXgm+d5wQKbjG2YCLDsAlltABAXA+v999DcJAAxBEARBSQ7AXppBRwGA5YfpRAFgu+bPTgDYSdNnM/B97733YmwGwk6aQtsBsJdm0FEHYLOmnskEwKo+heQkE7TbRFgcgPf0hAOyGQAb9f9VkQDLyzeAjQBYVfIrADAEQRAEpTAA20Fw2AAsN6WMNwDHq/+vl+ivGfiagbDXKHC8+gFHAYDNHvijCMB3797V7BSAg8oE7aUf8MN//nckUwsykd6f77Ps/xtUAiw3GaCdALDX6C8AGIIgCIJSCIDdRoFTEYDjnQDLKQCbwe/7779vaCsIjjoAGyXCigIAm0W9wgZgGXj1prpWrFgRSCZofT9gOwB2+zkkOVGd2T1q1DrD7t406pqgvzfl+1O+R91YXt5JiwmjbgPy/WLWYsLuPtHfI26bPzvp/6siAZYVAKtMfiWOLwAYgiAIggDAmR4MgwRg+QE6XgAcpQzQTps/W0V/Bex+8MEHMZYh2EkU2G0z6Khkgk4FALaCXacA7DYTtJNEWKr7AZvdp1ZdFMzuT6f98426KOhB2A6G9fPqwdcN/Bp1GXDaUkJ/j5hlSvfT/1efAMtL/1+zDNB2AOw3+gsAhiAIgqAUBmCrKDAAWC0Aq+j/KwOwUfRXQO+HH37ILUOwURRYFQAbJcKKwqeQEhWA3YKuUwBWkQladTNos2zQTltqqExSJ9+nehA2gmEj65cxasarKmGc2+iv3eeP/DZ/VpUASwbgIKK/AGAIgiAISiEAdhMFTiYAjsI3gMMCYAG/MgQDgFMLgL/55htuGYDDSoQVdjNoL02hzSDYKQg7sRX4qoJfJ9Fflc2fvfT/1SfActL82QqA/Xz6CAAMQRAEQQBgyygwABgADACOPgAL2DWyGwC26wesbwZt1A/YTzNofTIst1Fgu6bQZhDsBoTNoNio3Aza9HBmBL92TZ/9Rn/dZn+2a/6suv+vAOCgor8AYAiCIAhKMQB2GgXOgibQaAINAI4UAFvBrhUAu02EFWYzaK99gf1kbDe7X40+B2cEYXbAawZrXuHXrulzFKK/bpo/6wFY9E0XAEzXqxEAq4r+AoAhCIIgCABsCMEAYCTBMgLgKGSBTvYkWG5B1wqAly9fHql+wE6jwCqaQruFYCsQtgJip4BmBGNe4FdV9DdMADb6/q9ZAiy/AGyV/AoADEEQBEFJDsD6Jn9uosBZ8BkkfAYJn0FKSAD++uuvufUArOJ7wG6aQcsQLJrWOkmG5SQKbNcU2ioplhEEm4GwFQxb2QzIzOBXbJ/VfeGk6bOX6K9V8iu/zZ+99P81AmCV0V8AMARBEAQBgA0f+lIRgO0yzEYFgK0g2MxOor9RBGC5iWeUAFh++I83AAvg1dsPAEc5CmzUFNoKgvX3rRsQNusiYmcrCDOL+rqFXydNn6MQ/Q0TgJ1EfwHAEARBEJTkAOw1Chw2AJs9PMcbgFV+ZkVFP2CzKLAZCOvLjaK/Kvr/WgGwm+y2Vg/68QJgOfIVTwA2A10nAGz3PWCVzaCtkmF56QtslxBLFQQb/a00Aiqn1tclr8foZZAX+DVr+uwl+muV/EoGYKsWEH6//0vXKV2zMgCrjv4CgCEIgiAIAGwIwVEAYLsIUrICsNsosBEIm4Gvl+iv2wRYfj7vEm8Alh/84wXAbkHXCoCXLVvmKgqs/xySXTZo+V62epHlJQps1hTaqvuCEwj2AsJ2fzet5rcCX1Xw66Tps+rob1DNn70CsJvzAgCGIAiCoCQHYC9R4KgCsNEDtF0UKcoA7KQZtFMINgJhfbkV/Hpp/hx1AJYf+I0e+gHA4TSD9hMFdtsf2AsEyyCsh2E7ILYCXhl6zcDXD/z6afpsFv2VAdiqC0BQzZ9lAA4i+kvnBAAMQRAEQQDgTBBsBMBGD8pm0aJ4A3AUvwXspB+wvhm0XVNoMxA2A18nTZ+tmj9HMQO0UcQrigAsZ8I1yoYbNACrbAZtlQzLSxTYSUIsFRDsFoSNYNiJ9XXo12EGvn7h1y76a3Yf2EV/rZJfyQAsEqc5zf6s//wRXateANhN9BcADEEQBEEpAsBuo8BuAZic87Ec4sECDsD58+XLBMF6ELYyLY/jGKyzZ89uCAJhAPBXX30VYwHAfqLAZi+2VEWB3TaF9gLBVtFgKxA2gmE3NoNeGXzNor5BwK/Tps/xjv7S9UrXblDRXwAwBEEQBAGADd+eewFgWub/fXcHDsh0fEWTZTcgLMpxfsI5R2EAsB52jawCgN00g/YTBfaaFdoMgs1ac5hFg2UQNoNhr7YD36Dg167ps+rob7wA2G30FwAMQRAEQSkEwG4g2DMA370FB2Q6vmZ9gp0Y5yecc6QCgJ0ArhMAXrp0qeNm0F6TYTmNAqtoCu0Xgp2AsB6G3UCx0XL6up2Ar1P49dsHPojor1XyK7vsz3YArCL6CwCGIAiCoCQGYDfJXMwA2OwB2RSAv3kHDsgyAMsQbAfDohznJ5xz5BeAVcCvHoCjGgW2awqtEoKNosF6EDaCYTMgNrPR8vp1yOsPCn7tmj6bJb5ymvk5qOivGwD2Ev0FAEMQBEFQigGwUwjWA7CTTNC0zP989SYckOn46hNjGYGwmXF+wjlHXjNA073mB4C//PLLGDsFYKtkWHIUWEC92yiw/F3goD5tZtYn2A8IWwGxnc3qcgu+KuHXTdNnFdFfN8mvnACwqugvABiCIAiCkhyAvUaBPQPwl68p8e0rB9m6hZM0n09bH1O+a/Uc9vXbZ5Wtz60/fz2dffzKcT5Mv/vWzw98nTIAG4GwnVWeH9j8HIUBwHrYNTLVtWTJkphm0DIEiyaofqPAZt8FdpIQy64ptFVSLDMIdhoNNmsabQXDbi3XKUOvGfiaRX3N+vwG+e3rsKO/AoDpenUCwF6jvwBgCIIgCEpBAHYCwZ4B+PNXlHjd/Ins+eeq8l9ymxfqsfo1K7G/fXCVl3dv15TdvpSmbH1eto9Mw5cPbmCj+3cJfJ10fPWfSHICw6Jc5fmBzc+RCgB2ArhuAdhNM2iVUWCvWaHdQLCTJtFOQVgPw07B2GwZfd36ddtFfZ3Cr5d+v04SX/mN/srXuT76a9T82QiAVUZ/AcAQBEEQlAIA7CUKTMsbPRxb9QPmgPXZdSVeN388Gz+oW8w0GhfTjm5ZxH66dZ4PH9w4n/Xu2JzNnzhYm0bet3YO69+lFS8/v281LyPT8KZFk3jZS0c2avN//cZJNnlYTz79jfTt2nSan6ZNH9WHL3/74j7WufXz3FT2+avH+K+Yn+qm7dm2bKqy40EWACzbDIaNrPL8wObnyCkA6/v/CgBWAb96ALZKhhVWFNiuKXSQEGwGwk5g2A6KrWDXDnqdgq8q+FXR9FlF9Ncs+ZW4Vu0A2E/0l/4fAoAhCIIgKAUB2A6CvQLwf39yVYnXzRvLxg3sEjPtx5tnWMH8uflwlWdLsvcv72OX0tawzq0asbfP7GCLJg/hw1S+bPpw9mKT57TpOR59mM9PzvrAn9m2pZPZtUPrWZGC+dhXrx1jf71znpUtUYTtWzOLT69Qujivm+an6a+f3MrLqP7PXj7Mhvdqx031p+9cyto3q8/X27tDMza6X0c+vXubJnzdqo4JHd8333yTWw/CVjAsylWeH9j8HHlNgKUCgL/44gvNVNfixYuVR4HdJsRykxVaJQTbRYPtQNgJFDuBXTfg6wR+xX5bNXt2Cr92TZ9VRn+dJL+i65SuVxmAVUd/AcAQBEEQlCIA7DYK7BmAP7qkxOvmjGbjBnTOND33Y4/w3yrPlmDvX9zDh7994zhL376Yu0iBfHxaqaKF2F9vndGWq1+9PJ+f/ELdqtr00X078OV2LZ8as75rBzPAumUD9vKR9Xxd9PvjO6dito9Mw7R8+2Z12X+8f15bP5nGv3r1iLJjQsf3jTfe0CDYCoaNrPL8wObnKAwAlkHXzHYArDIKrKIptFVSLLvEWG5B2A6G7YDYqfV16tfpBny9wq+bfr9Omj4LADa7zt1Ef2UAFtFfGYCdwK/b6C8AGIIgCIJSGICtIFgGYKf9gDlgfXheidfNGZkBpB1jpr19ajOrUvZpPky/71/YyY5smMNqVCjF5yVzQM4oL1W0YMyy7ZvW4fOTaVhMp2XSty/k61s1Y5g2neZ7rnIZPrxr2eQMGK7PypYozAZ3a6ltH5mGaXmq86tXDmrbF4QFAAvrQdjOKs8PbH6OvPb/1QOwE8h1CsBhRYHdNIVWBcFuo8F2IGwEw35sVL8d+Dpt8uwFfu36/fpJfKUi+kvXrBkA+4FfADAEQRAEpSAAu4kCewXg//v+GSVeO2sEG9e/gzb+5Y39rMIzT7ETm+bw8cpli7P3zm9j7V94jt04tIpP++jSTpY7xyN8+IU6ldmR9TP58A9vHmHZHnqAzy+WEfXSOtK3zmcX9yxhjWpV0KavnD6UTRrUmc9/df9ybXr+3Dm07Vs6ZRAfpuVFncWffJx9dnUPH37rZMZxmTda2THRA7CR9dArl6k8P069d8VkfqyMpp/dschRHb+8c5wtmtCf1av2LKteviQb1bstvx7C3hen58gvAPsFX+EHH3xQPOjDcEKZrl0/TZ+tor8AYAiCIAhKYgC+cuWK5yiwZwB+L12J184algGz2TjoVij1FPeRdTO0cg7A57ZmAPFsVuqpAhkAWpub5qPyz67s4sM0H4FT7cql+fxkmk/UM65fe5a+ZS4fHtW7DQesFvWr8WV+eOMQN03r3roBh+phPVrxeT+6uJ0VKZCHLZ08gC8v6jy7Y4G2PbT+t06sV3ZM6Pi+/vrrmu1gWG+V58ep6fj+7re/ZRd3L840nc6x3fJ0/Ok40vG/cXAlP56zRvbg18a76ZtC3x8n5ygqACz7888/9+TPPvvMkT/99FNX/uSTTyz98ccfO/JHH31k6A8//NDUH3zwgaXff/99Q7/33nua6TyJ4Tt37pj69u3bhr5161Ymv/vuu6Y2+7a3VdZ3s1wAsq1ajBj9DZH//gi/9tprhn711Vcz+ZVXXomx/lNHQSW+AgBDEARBUAoDsNMosABgN/2AOWDdPhG6f3gtjX10fmvMtC+v7Wa/vHlIG69XtSz77PIO27ru3tibqS4yTaM6nWwPrfe9M5vYP24eVbqfdHzpwdLoIdQMiuWyeJyfcX3bsn4dmrCC+XLyYytPXztziDb+7qkMsB3RjVs+TzRf91b1M9W7cc5wVr18CT5M85/dNo+9dnglmzSwA1s5dVDMuQ/TMgC77f9L95oqAPYKvG6gN4rgGwb8ygAcBvyaAbAb+HULwGHArxUAB5H4CgAMQRAEQSkCwF6jwF4B+L9uHY2ED6+ZwsqXKsIWjuvF+rV/nrVrUjMy2+bVAoD1tgJiPQCHvc1j+7Zha2cMZnNHdWcvPFcx03QavrhzHivyRG62dGJfPl/BfI+xN4+u4mU0fPPkGsO6s2X9C/vy6g52atNMVrxQPtaoRjleZ682Dfm5//s7h+JyjsIGYD+w6wV6gwDfoOHXDnzdwK8AYFXw6yX6GyT8GgGwG/g1AmDV8Osl8RUAGIIgCIJSGICdRIFpeTk5jpNm0Bywbh6KjD+9sIntWDiSnd0yM1Lb5dV0fOlh0urB08rxOD9j+7Rma6cP5MONajzLlk7onWl6+ZKF2cXtc7Rl9iwZw1rUq6Tts1ndlUsX5ef21MZpLHeOh9nf30rTymhdacvHx+UceW3+bAXAKiDXD/B6gd4ogG8Q8GsHwMkGv8kW/QUAQxAEQVAKALCXKLBnAH47DQ7IAoD1dgXAIW/z2N6t2Nqp/fnwlxc3sfy5HmXX98yPmZ4t659jlvn++g4+Hw3nyPYg+/TcBsO6C+bNwe6cXMVOrZ/C2j1fPaZs6fiefB3xOEd+AVg17IYNvarBNx5Nnq3glyDXDIABv69GPvpL/w8BwBAEQRCUggBsB8EyADttBs0B6619cECm4ys/RBo9bFo5HudnbK+WGaDbTxs/u2kqK14oLxvU8XltOoHslxc3aPO8un8BK1OsAB/u1uI5NnVg20z1Xtw6gy9Hw6fWTWa1K5TItN6Fo7vG5Rx5af6sGoC9wK5f6E0k8PUS9ZX7/BoBcDzgN0r9fqOe+ErALwAYgiAIglIEgN02hfYKwP/5xu64+Pqu2WzNlD4xPrNhsrL6f7q+jV3YPC1u+0fO8uu3MY0iKkZgrJ8ej/Mztmdzfi7kaRP7tmJ//Pffa9NHdmvKujWvxf726g5+nBtWK80WjOrCyz44uYJHgZeM7cbLadrJtRP4NPoV41TfoeVj+PgbaQt4+e1jy+Jyjrz2//UKwH5gVwX0hg2+8YBfPezqATjR4dcpAPuF36CaPruJ/gKAIQiCICiFANhNU2gjALZrBs0B6/UdcfHYns1YpybV2JrJvTSfWT9RWf23jy5mlZ8pHLf9IwsA1tsOiGMAOORtpvNwaNmomGl/e2Ura9eoijadxge2b8BKFs7HihXMzab2b53p2NP8ObI9wF27fHF2YfNkrfzkmnGsdf2KrG+buqx8iYK8Hv06wzxHXps/WwGwCsiNB/S6BV8/Ud+w4FcPwEHDL5o+q43+AoAhCIIgKMUB2CwKLADYTT9gDlivbouLx/ZoytZM6plp+hv75rBXds/Sxt85MJ9d3z6dD59ZN571fbEOmzmoDfvu0jo+jcpofqqP/M35NbyM5imYJzvbMXtA3PaRji89GFo1IbRyPM9PkD65eixr17ByJLaFjrFV9NcKgOleo+WDgF0VwBtP8A0y6usWfmUABvz6z/ocRtNnADAEQRAEpTAAO40C6wHYSTNoWuY/XtkcF4/p3oQN7diAnVg1SvOtQ/PZ63tnsVrli2nztWtYiZ3fMJ4dXT6CNaxair2yazrbPL0Pq1vpaa2e6mWKsMubJ7Hl47qwJjVKsx+vruPzFCuQiy8br30UAKy3GwCO17YHaTrXdF6jsC12AGzV/Fk1AKsCXjfQGzb4xgN+BQADfhMr8RUAGIIgCIIAwJYQ7AaABQRzwLqxIS4e060xK//0E6xdg4qad8/pz8uqlynM3j8yn/14eTUr81R+Pq1hlZLs6LJh7NbBOdyVSxVin59awuvZv2CgVi/VSb9innjtH5mOr3gYNHpotHM8z0+Q/vrMMvb67umR2BY9ABtFf4MAYJWw6wV6owq+fpNduQXgqMGv26RXydj0GQAMQRAEQSkEwJcvX/YMwbT8ihUrXDWD5oB1fW1cPKZbI7Z6fGfDMpo+pU9Ttn5SV7ZgWBs+rXLJgmxoh7p8OeGvTy/mvydWDNOWpfno99aBmdpwvCwDsBfH8/ykip0AsFnzZzsADgJyw4ZeJ+CrorlzEFFfuc+vEQDHE34Tsd9v2ImvAMAQBEEQBABWAsByM2ha5h/XVsXFY7o2ZKvHdTQs++HCUlbmqXysboVi7Kv0BXxa35Y12K5ZvbV5zq0ZodVzYtkQbXqlkgX470dH5rDyxR+P2/6R6fgaPQS6AeB4bn8qWACw288fyQAcBuj6gd54gW9U4NcIgFMNfhMx8RUAGIIgCIJSBIC9QrARANs1g+aAdXVFXDymSwNWIPcjHFiFp/RuopX3bFaVta1fThv/7NgcPk/zms9kgHFRNqJjXa2eE0sHafNxAP51uNazRVjXJpXjto9mAOzUXs8PHbu3dk/2vN3vHZjOjzMNp68YwhYPf5EP3z2zkO2a1VObj8qWj2obt+Or6hx5bf4cFgB7AV6v0JsI4OsEfvXRXhmAg4Zf9PtV1/QZAAxBEARBKQ7AVgmx9ADspBk0B6zLSxPK7+2fyr46MSchtpWOr9lnqxwDsMt1nlgygD14/x9Zz6ZVPG/3u3smsUolnuDDB+f1YSM61Mk0nbx6TDvWtt6zCXcN6c+Rl+zPQQKwV+D1A71hg2+Y8CsA2Ax8kwF+k7XpM5n+JwKAIQiCICjJAdhLFFgAsJtm0BywLi6CAzIdX6vM3Xb2cn6a1yjJDs7txXI/8gD74fRcbfpbO8axa+uGa+MfHZjC0pcOiCmf0qMhWz6iNXt96+gM0H08Zj76ndGnMSuQKxtbPboNu3tyNv9tW7es6bZ8dWwmmz+oGa9XXjf53d0T+HQqp7q2TelsuBzNR/tD81DZL+cWsE0TO7IxneqyfbO6Z9q/lzeNzLQuu3Pktfkz3Wt+AdgP7IYFvU7ANwpRX6vPHJkBsBX4RhF+o9rvN8joLwAYgiAIglIEgN1CMC2/fPlyV82gaZm/X5gPB2QZgL3Y7fn5MG0SK5z3UT7c9fnybNWo1loZDY/u9Jw2fnxR7wx4Lc2HL64exPLlyMrmD3yBze7XmFV6+nFueb6bu8ayPs0rc7Cmer48Oo3XKerQ+9NDUzJg+WE2uUcDtmx4S1ayYE62c1pnXnZj4zBeD61r0ZDmrFqpAnycyqhe2ocR7Wvx5WqVKcS3jdZP5c1qlGA9XqjI1924anFtn2i8XNG8rG75wnyb3Zwjr82f3QCwCtBVAbzxBN94wS+BrhEARxl+0fT5n/ALAIYgCIIgALAhBMsA7DQbNAesc3PggJzF5NNWrgDYxfomd6/HTcMXVw5gZQrn1spWjWzJRnesrY0fX9CTta3zDB+uVuoJdmRed61s59SOGQCcP9N8N3eM1qaLOkWZ3iPa1WSz+zbSxu/sHpsBuX/hw61qlWSbxrfVys4u66uVzR/QhA1oWUUr+/LwZHbfH/6Nr5v2ibZVlP18ehZf7vuTM/i2yPvr5hx5bf4sAFgl3EYBeqMEvl6bPOv7++oBON7wmwj9fo0A2A38+k18BQCGIAiCoBQEYDdRYCsANosCc8A6OxMOyFksvu3sxG7PD8Fg14Zl2egONbnv+8O/shvrBvGyVSOa82li3uPzu2fAayltObme709MzQDdfJnmu7l9hDZd1CnK9K5VpgBfVr99n6aNYwVyPcTr0pfRL9V3YFbnmDJaJ81P66P5aFz4/j/+QSsb0rqqp3Mk4Neq+bNR9DcoAFYBu4kAvkFGffXJrmQABvwmTtNnADAEQRAEJTkAX7p0yXMUWACwVTNoPQTTMn87Pd2Xd05uyz7ZN8bTsn2aVuC/V1b1YxeW9+HDN7cO0+r7YPcodmphd9/bGC+7falhBMBO15U2sxOrVDwfWzm8meYejctxUzmND25VRZt/77QOrM1zJflw4TzZ+LEWZW9uGsLrouFj87pq89G5EdNFnaJM71Y1n+brEOM/nZzKcjx0P/+tVbqAdr7Jt3eMYLkywJaGaRvn9WsUUxeV0bq3TniRdckAfKP10baMbl/D0zlyGv3VAzDda34BWDXsBgW9YYKviibPegsABvwmVtNnADAEQRAEpSAAO40C6wHYSTNoDljpUzz79vZhLNsDf8wAj+qelq9UPC//XTsyA9iGvcCHqa5jczvzYfptU7uEr22Mp8U5NTuvdnZzfhpXLsLSZnSImfbNoXEZ0Pkn/vvq+v4cdL9IG8N+OjGJ1Xm2oHZs6Zh3qFuKT//u6ATWqkZx7dzI5+CTvaNYrmx/5vXROJ0zWu/NrUNiTHXsndqWlX4yJ18fzTu+U01erzjf5Z7KzeujcppO9VLZ9TV9Wb7sD/Dtpe2hbbv/j7/n9dK8NB/NI66/wa0qa9vi5TrUA7DT5FduATgo0PUKvUGBbxhRX7fwKwA4WeA36v1+VTZ9BgBDEARBUAoAsFcIpuWXLVvmqhk0B6yTEz17dLuqbOPoZqxY/kfYT8fGa9PTprXVhj/YOZRdWd6TD9M8i/o3YD0alWFnF3ZllYrl4dPf3NCfm+ZrUL4QG9yyIh8+NqdjBnw9bbjubw6MZrN61mF9mjzLTs3rbDj9+qreMdtCwzR97YgXeP20bTT9k93D2fiO1bVlRD1U7+GZ7bXtd2v5nHqx0/Pz3eGx/LjJ50CY9ov2gYYnd6nJzxUd93l96vFxcV5o36ms3FO52N4pL/JxKqN9F/OJ+grneZjd3jaYH0+qS29xvFYObczrLFkgewZgl+THVNRD14FYH10LHICl80T1UDnVQcM3Nw/kZTQvLSPWReNiGZrXyznykvxKBuAw4FYF8LqB3rDBV3WTZ31/XzsABvxGN/oLAIYgCIKgFABgL02hZQDWN4M2iwLTMn89Ps6TfzwyhpV44lH+279pObZnUiutrFKx3Nrw0Vnt2ai2Vfhwm1rF+fBLK3qw3o3LcpCi6SuHNOJ+Z2M/Ps+sHs/xYVqWxo3WT+tYO7wxr6tBuYLs0Iy2fHqdsk+wub3r8uld6pdixfI9wqfT9tV8Jj+fvmHUCxy4qP5vD45ipQvlYGlTX2QXFnfl9b6xrg9ff46sf+L79v72QZ6OER3fixcvZrIbAPZ6fqJsOrZkMf7K6l78HNDw53uH8XMkX2d0Hug8BbEtRgDsJPmVuM/CBuCoQW9Y4OsHfu0AOIrwi36//7T4W4hHBAiCIAhKMQC2iwILAHYTBeaAdWy0J++Z2IINblGOD7+yqgerU+ZxrYwD8K/DR2e2YaPaVGKf7xmcAaPZtOk/Hh6pja8c3JCbhmleWkYs26ZWsUzrPjO/A+vw3NPa+K3N/VjNUvn4r7wd3x4YzgrnfogP03QqF2Vd6pXk9e8Y34x1b/gMe2dDH+65vZ9j07rW4MPyfnixGQA7tZ/zE2W/sbYXK5DzQTa8dQU2q0ctfh3Q9URlX+8byl+s0Pmhc1GtRF5+TQS1LXSMvUZ/6V4LGoC9AG9UwDfo5s5W4Kvv72sEwCrBF/AbTNNnADAEQRAEpRAAu4VgPQA7SYbFAevISE8moCxdKDurVDQX94N/+gO7tbEPL6NxMd/RGS+yUS9WZO+s68UaVygUU4eYb+Wg+tw0TPPSMmLZNjWLZlr3llFN+HzytBJPPMIuLOjIWlYrEjOdtlGs69v9Q7Xpw1uV5/XTegmeqT5hmk7ba7RuN6bje+HChUx2BcA+1h9lf75rINsw4nl+/OlYy2U/HhzO9kxozsvOzG0f6HboAdhp8qsgANgP7LoF3niCb9jwawTAqQi/Uej36yX6CwCGIAiCoBQCYDdNoc0A2CoZFgesw8Nc+9aGnhnQmDdm2sqBddngZmX5MMHo5zv78+Gl/etkQGUF9uOBIaxYvofZ13sG8OlvrO7K8j36F21ZMg3TvIemtuDDR6e3Ym1qPGW4/nKFH+N10vie8U1Zl7pPs2/3DWIFHntAW0fapGYs18P38+H+TUpr66D5CufOyuu/vKgDa/DsE1rd76zrzt7f3Jv/Gq3bjen4nj9/ntsIhO3s9fzA7s6Rl+RXfgDYL+jGA3oTFXyNADjZ4TeZmj4DgCEIgiAIAGwKwbT80qVLXUWBaZlfDg527VGty7ENw+rHTLu7pz+Hzx/2D+RlRfM+xFpWfZLVKZ2Pz0/zbB/9PJ/epkYR9nz5Aqx0wUf59BUD6nDT8NFpzTmc0rw0nO0v97GKT+XU3LtRST7f3B7V+Titg37f29iDT989rjFfvmrxXKx7/ad5GU3/bHsfPq1c4Rz8t2mlgrx+KhvWoiyfRnXR70dberG313Tl2+nl+AjLAOzFXs8P7O4ceY3+GgGwSrhVBbx+oDds8A0CfmUATib4jXLTZzfwCwCGIAiCoBQGYLMESU4g2AqAzaLAHLAODAzEn23rxd7b0C3T9K929mG31nVVsg6juuTxu7v73oPsX7eH5hdl3esXZ+fnto7ZXlXbJUzH99y5c4a2g1+aJ8jzA//zHHmN/goAjiLwhgG9TsA3nlFfPQADfhOv6TMAGIIgCIJSAICdQrAZAOsh2CoZFges/f2SyoNeKMWeL/c4W9K7BqtZIjdbM7A2n35+TktWIv/DbG63KmxY89K8LOhtsQJgJ07G8xM1CwB2Ev2Vk1+R6V5TCcB+YFcF9EYRfFXAL0GuGQBHFX6TKemVX/gFAEMQBEEQANgQgs0A2CoKTMv8vK930vnaglZs87Dn2GvL2sRM/3BDZ7Z9RF12cGIj9v3unoFvBx3fs2fPGtoOfmmeZD0/UbIVANtFf70CsArQjQf0qgZfP1FfN/BrBMBm4JvK8Bt2v1+nACwSAuIRAYIgCIKSGIC9QDAtv2TJEldRYA5Ye3rAAZmO75kzZ7jNQNjKOD/hnCOv0V8jAFYNt0EAbxDRXpXgqyrqK1sGYK/gm0jwm0zRXwAwBEEQBKUIALttCm0FwGYQnPX++8SDBRyAs/7lTxoAezEtj+MYrO+//37P0V8BwIkAu0FFe8MEX6/wKwOw6qgv4Dd4+AUAQxAEQRAA2BCCBQC7jQLLn0Uii4RAlBl3/fr1mjds2MA2btyoedOmTWzz5s3cW7Zs0bx161a2bds27u3bt2vesWMH27lzJ/euXbs07969m3vPnj3ce/fu5d63bx/3/v37udPS0jQfOHCA++DBg5oPHTrEffjwYc1HjhzRfPToUe5jx45pPn78eIxPnDih+eTJk5pPnTqVyenp6Zl8+vRpU/sBYbJV3UbbYrTN8j7J+6o/DvIxEsdNPpbyMRbHXT4X4vzI50ycR3FexXkW511cB/K1Ia4Xunbka0lcX3StydeeuB7p2pSvVbp25WvZ7rNHTqO/dK+pBGDVsOsFeoMC3zCivlZZnuk8hdnkOQz4TdR+v26aPgOAIQiCICjFANgNBNPyixcvdh0FFhBsBMAyBBNEyBBMkGEEwQQlRhBMEBMWBMsgbATBKkHYDIbtgNiPzdZntn1+wNcMfuVjHTT8ygAsw68MwDL8ygAsrlsZfs0A2Ap+VQJwUKAbBvSGDb5+or5yk2czAAb8RhN+AcAQBEEQlMIA7BSCZQBWGQUOA4LNADgICPYSDdaDsBUMWwGxU0B2urzVNui3V78/bqO+quHXCoDjAb9eor9mABwG4PoF3qCjvWGBrxP4NQLgoPr7Jjr8Rq3pMwAYgiAIggDAtgCsIgosIFgPwG6bQgsI1jeFDgKCvTaJtgNhq4iwExh2C8VeYNcMeu0ivm6jvnZNnoOEXwHAevh12/RZALB8rXuN/tK9RvddPIA3DOhNBvA1AmDAb3Siv07gFwAMQRAEQUkMwBcuXPAMwXoAtoNg8aAvfxYpjChwlCDYDwibwbBTIFZhs/XbRXudgm/U4Dfo6K98T9jBbzwA2CvwuoXeoMA3XvArA3BQya6iCr+J3vQZAAxBEARBKQzAdhD8wAMPIKsvDIed6Ttr1sjBrhfg9Qq9UQJfqyzPdK4Av4kJvwBgCIIgCEpyAPYDwUafR9Lb6MHF6AHH6oHIrO+Y0cOW1cOZ2Sc8zB4ArR4azR40zR5M7R5orR6GnTShNHsQJ1t9o9QpCJDtoMIJmHiFHrIXyPILdrLj1fQ4SqAbZehVCb5eor5yk2c7APYKvoDf4Pr9AoAhCIIgKIUA2E9TaDsABgQ7h2ArEFYJwTIAuwVhO8BwCsJ+YNgLeAUBxCrhOH/+/IhwJ1mkPpngN4ykV0b/E8L45JHe9P8QAAxBEARBKQzAgODEgmAnIGwEwKqjwWGAsF8YDhKI3ZrOSVS2xU+E12+kV3XEN4zmzn4TXYXd5DmK8BuVps/i/yEAGIIgCIJSAIATGYLNso5GFYLj3STaDIATGYRVAXG8oDjeAKziuIUFvarB109zZ8Bv8sAvABiCIAiCUhCA49EfOFkgOIr9gs1AOMuvGWrtIl7xAuEowXBYYBwWAKs+JirOk0roVQm+KqK+UWryDPh1Br8AYAiCIAhKYgA+f/68KwgOMwoMCA4uGpxF+kZpPEA4HjAcBBCrhGOVABzkfqo6F6qjvWGCbxhR30SE36gnvXLS9BkADEEQBEEAYEBwkkCwDMJ6AFYNwUGBsEoYDhqI3ZrOSZS2RyXseoHeeIFv0FFfP02eUwF+49nvFwAMQRAEQSkCwFGHYKsHqGSC4DATZBkBcLxBON4wHG8ojjcAB3EsowK9yRD1BfyG0/QZAAxBEARBKQLAgODkgGCnIGwFwEGBcJBR4aBgOEwwDhOAgz5WXs5fooNvPJo8A36DgV/6fwgAhiAIgqAUAOCg+gNH7fNIUYHgeDaJpnNvlyk6CiAcVRgOApBVAnDY++7lHLmB3mQH3yD6+0YVfqOa9EqGXwAwBEEQBAGAQ48CJxsERy0aLADYySeTnEBwGCDsFYbjBcRuTeckEbYz6tCrAnyjEPUNE36t/hYmCvyq6Pcr/y8EAEMQBEFQigAwIDj5INgIhPUAHG8QDhOGowrEUQdgP8fb7bkNCnzDivomS3/fKMJvGNFfADAEQRAEpRgAA4LtQdjsITOeTaLdRIPNADgKIBw2DEcFiqMEwCqOZ5SgVyX4xivqmyrwG++mzwBgCIIgCEpRAE7GzyOphuCw+wWrjAbbAbCq/sFhR4VVAnHYYBwPAFZ5nLyeq2QB30Rq8gz4tYZfADAEQRAEJTEAnzt3LlAABgTHp0m0HQg7AeAogbAfGFYNxEHBcVAAHNS++zkffq6DRAVfwG/iwC8AGIIgCIJSFICDjAIDguMbDaZz7+bbwVECYb8wHCQQ+4FlpwAcj22PJ/CqhN54NHcOsskz4BcADEEQBEGQBwAGBCc3BBuBsABgN98OdgPCYcKwCiCOFxTLpnMS721QAbuqoDfK4BsF+LXKQxBV+I1Xxme38AsAhiAIgqAUAOB4QbBdts9kgeAgm0R7AWE9AEcBhKMEw/GA4ngBsMrjFSb0pir4eon6An7dwS/9PwQAQxAEQVAKADAgWA0EJ0I02AyAkwmEgwDiIOE4KAAOav9VAa9b6FX9HV/V4JtoTZ4Bv5nhFwAMQRAEQSkEwG6TYqU6BEetSbRTELYDYLcQnCgwHCQQ+wVmNwAcj+1XfR6CgN4gwDfqUd+wmjyHCb/xSHolwy8AGIIgCIIAwHHPDK0Sgs1AOEwIjnc0OGvWrOIBD46Q4wXnYQCvF+hNBvAF/EYHfq0AWP9/EAAMQRAEQUkKwGfPngUEJ2i/YJV9g51+PzioiLDbqHBQ0eF4RovjBcBBH8MgoTfe4Osn6pso/X0TEX79Rn8BwBAEQRCU5ACcSBBs9gmNqEOw3ybRqQTCUYXhoOE4SAAO+9hEDXrDBt94RX1TBX6DbPoMAIYgCIKgFAFgQHB8IVhFNFhVkiwnEOwVhMOC4XgAsV9gdgPAUds3r+fI7bWQ6OAbzybPgF/n8Ev/DwHAEARBEJSiAJxI3wiOEgQH2SQ6StHgRIHhKAKx3nQ/Rn0b/QJvWNAbBPiGEfUNqskz4Ncd/AKAIQiCIChFAFhVFBgQHP1ocKKDsF8YjiIURxWAVRxnL+c3WcA3qk2eAb/nAMAQBEEQBAAGBLuF4Hg3iU42EPYKw6qAOJ5QHG8AVnX8vAKvH+hNZPCNUpPnZIJfP9FfADAEQRAEJTEAnzlzJuEhOKzs0PFoEh1WNNgtCEcdhlUCcVhwHBYAB3Fc/JynoKE3CPANurlzFJs8m8Gv2UvIRIZfADAEQRAEJTkABwnBdp+qAAQnbjQ4TBD2C8NBAbFKUPYLwGHun99z4fdaiCr4Jiv8uon6eoXfML716xR+AcAQBEEQlKIAnOoQHLV+wckCwipgWAUQhw3Fdqb7MUrbowp2VQCvG+gF+AJ+/cIv/U8EAEMQBEFQkgOwmyhwIkJw1PoFRyEanCwgrAqG4w3FUQBglccx6tAbNvgCftXBr+qMzwBgCIL+f3t3Ah9Vfe99PGSHBLKHhCwECARCgEBAwqYgImoV60LVQqtWW616tdflaotVq6Vqa+VqL4pKLRXrUuujt7ijIu7U+tTWKm51ARV3pFDZOc/5DvnzHKaTZJYzM+fMfN6v1+9FMpnlzDkzw/nOfwOQpgGYEOyfLtGJDsJ+CsNuB+JEheNEBeB47Rs3j58fgm8iWn39MtmVH8NvV62/BGAAANIoALs1HpgQHH4ITkRrsFsTZSUiCLsdhuMZiN0MyrEG4EQ+P7ePTyJCr9eCr1dbfQm/BGAAANIuABOC/dUl2k9B2AthOFmhuLvS+9Fr2+SVwOu34JvMVl/Cb+zhlwAMAEAaBmBCcHJCsJeCcDy7RkcbhuMZiJMdjL0QgOO9bxMZer0YfN1o9U10l2cvLHUUTfiNdNIrAjAAAARg12aG9mMIjleXaDdag93sFu2VIOzVMJyIcKxjkMgAnOh95sXQ67XgG89W33iM90318EsABgAghQPwo48+mpAQ3NmSFYRgb7QGuxmEkxWGkxGI3QjJc+fOta6//vo9AXj58uXW+eef7/lg62bgjTX0JjP4JrvV189dnr0afgnAAACkeAAmBMevS3SyW4P9HoRjCcNeCcTd1e23325lZmYGTrjb2tqsrKws65ZbbvHFtscSeBMZer0YfJPd6pvu4ZcADABAmgdgP4Zgv4wLTkRrsNeDsBfCsJdDcUFBgTnhtmbOnJmSYdet0Ot2a69Xgy/hN3nhV/8fEoABAEiDABxNCHZrjWA/hGCvd4kOJwTHKwgnIwy7FYi9EIqnT58eOOHOzc1NmaDrZuCNR2svwZfw21n4JQADAJBGAZgQHHsIJgi/lJQw7GYgTnQ4VjdovR9vvPFG3wZdtwNvpKE3mcE3VVt90zX8EoABAEizAEwITm6X6FQJwskOw/EKxPEKyPPmzfNFwI1n4I1n6PVz8CX8Jjb8EoABAEjhAKwZZ2MJwYMaKveMXaQoiqIoL9SA/v2iDr8EYAAAUjwAdxaCuzphMCFY92G9/j2KoiiK8kzp/6Zowy8BGACANAjA0YZgAjBFURTl1QAcTfjV/4cEYAAA0iAARxOCCcAURVGUFwNwd1/gdhZ+CcAAAKRRAI40BBOAKYqiKL8F4K7CLwEYAIA0C8CRhGACMEVRFOWnANxd+CUAAwCQwgH4kUceiSkEE4ApiqIovwTgcMIvARgAgBQPwLGEYAIwRVEU5YcAHG74JQADAJAGAbizENzdGsEEYIqiKMrrATiS8Kv/DwnAAACkQQCOJgQTgCmKoigvB+BIwy8BGACANArAkYZgAjBFURTl1QAcTfglAAMAkGYBOJIQTACmKIqivBiAow2/BGAAANIwAIcbggnAVKS1+W8nWc/d+fVA6Wcvb+v2V0+21q6c82+Xr3/hhEDFct8P/foQa90zc/fcX2fXe+fx4zy7fz59/tuuHsONfzkxsF94n1DxCMDhhl8CMAAAaRqAwwnBBGAqkrrvhoOshpre1tEzBwZKP//h2hkx3Wd7a2XctlfhU9sYfLm2feo+/WK6b91+xdJDAyFS76M3Hzl2z307Q6+X32MnHDkk8Bzc3N+x7leKChWAIwm/BGAAAFI4AD/88MMxhWACMBVuvXzf0YEw6Qx3agHVZSb8RVOhAmo8A7C2uXVYmTW2pcJa/eA3Yg7AZt84LycA836h3AvAkYZfAjAAACkegGMJwQRgKtyaO2uwtXj+vv92ubpCmyA5ua1qr7+Z39VFWC2jjf37WEMHFltLrpga6JqscJqdlRn4V/ej6151fnvgerrs2K8NCnSr1eX/PW+ide5JIwMtxlXlvQK/L/rJFKtlcGnguvded2BYAfiKc/fZc1vdX6jtDfW77t9s/6nHNQf+ZsKjuZ7+zc/LsmqrCgKP0V0AVnDWbbSNCuRP3TZrr9Z2PZYeU/tej2n2kfa3rq/bKXDqsczjaX+Zfa3S83Q+H/1+wMSafwvA2hcq5/48Y+7wwDao/nLvUXuO909/MC5wbHT/ur7ZbgIw5XYAjib8EoABAEiDABxtCCYAU+GWQlB3Lb3BYdP8fsl/tAVCkwloClYanxt8mzsWTLcOnVa/Z1yqwqpCmrkPBTfdTveh8HXy7KF7AmGoluRQAVjPQ92WdR/6m3MMbGfbr/vX45nxxI8u+VrgvWPCo/N24bYA63F1n6b12GyrWqhNy7q5HwXMwl45ex5PoVPboJ+1Tbof7R8Tav/nokl7HkOh14RXhfMLvte65zk7A7B+NvtTx1nX1RcVJozruOhnXb+8JH/PdpueAfqSgwBMuRmAow2/+v+QAAwAQBoE4GhCMAGYCrcUsrqb0KmzAKlAdtCUusAEScHdjp23+foBDXt1yVXYVWuvCcCmldO0SDuvG04AVpBTy6X5XYFPobu77VcQV3XWBTqaAKwAq5Za52VqgVUru4JncOu0rqvHU5gNbqlWq7kJwAqnup4pfdlgtl2h1hn4gwNw8P40re86DuY56jrmSwlTegyNBScAU/EOwOGEXwIwAABpFIAjDcEEYCrcUgAL1c3YOaNyZwFSAUqhTt14FZBUJogFh8fOArICnmmR7CywdReA1Yqp7sm6zJRzEq6uWrCDu38rSMcSgBUYg4OkHsd00TaBNvj5qjVYXxQ4/6Zuzeb6ainWz84KtZ3hBOBQ+0LXUSuy828XnjYmcGwIwFQ8A3C44ZcADABACgfghx56KKYQTACmIln2R+NOTaugCbZqjTRjQNVaa7o2mzG+pguxuh07x6I6b2MuV7BSa6b5XV1x1WXZjQCs7VHX4eDr6DLTnVfbYoK5Qr3GJ5uw6myt1T7QdTsLwM6u4p29x3QdPbbZXyq1kqtlWONsFczN35yPZ352Lr2kYG8CsO7T+fimu7ebAViTiDm3W6+LF+4+ggBMxS0ARxJ+CcAAAKRhAA43BBOAqUhKIUsBS62UCqoKQj84fsRegUpBUS2SaqU0oUldjxXo1Iqq1k2FWhPKNK5XLcMKTwpruk8FYXWb1vXMurKxBmDTuhp8HT2OuvCaAK7t0XZq+4v75O4V9NVlWs9NIU/PJ1QA1n0oyJq/6T2mbXWWaU12Pp72W3D3bD2Otjv48czkX2p51XbqtiYAa39pv+l5mWNk1it2KwDr/hW6zX4y2+0MwAry6o5tQrj2pXO2bIoKNwBHGn4JwAAApHgAjiUEE4CpaJZDUohVuDKzEjtbhNVNWsFIocf5d91Ot1F4c7YGq8VVtzEhTcFY43J1PWdXYv1srmNalZ2toMHbYu7bXK6xs87rO1tz1XrpHCesx9ZjOe9Tz00twfqbWpOdjx98PYVQM/GUczyuKWc3b7WEa3+ZoO8sbZf2hfZX8JhntfLeetX+gf2isOsM97pM26m/d7WPnM+hu/1pfjdjgHV9PYb2l3N/m+et/epcI1o/O8cfU1Q4ATia8Kv/EwnAAACkeACONgQTgCnKe6WgqJZVE5QVdjUJmWk1V6u7CZ4KraYLciK2LdQkWBQVjwAcbfglAAMAkCYBOJoQTACmKG+WWlzV7VrhVv86W2TVwqtuz/qbujg7u4bHu9TCG6orOUUlMgB3FX4JwAAApFEAjjQEE4ApiqIoPwXg7sIvARgAgBQOwA8++GBMIZgATFEURfklAIcTfgnAAACkeACOJQQTgCmKoig/BOBwwy8BGACANAjAnYXg7tYJJgBTFEVRXg/AkYRf/X9IAAYAIA0CcDQhmABMURRFeTUAd9aTqavwSwAGACCNAnCkIZgATFEURXkxAEcbfgnAAACkWQCOJAQTgCmKoii/BOBwwi8BGACANAzA4YZgAjBFURTlhwAcbvglAAMAkKYBOJwQTACmKIqivB6AIwm/BGAAAFI4AD/wwAMxheCy0mJzokBRFEVRnqjS0pKowy8BGACAFA/AsYbg7tYKDlXLly8PWY8++min9dhjj3VZjz/+eKe1YsWKTuuJJ54Iq1auXNlpPfnkk2HVU0891W09/fTTXdYzzzwTUT377LNh1XPPPRd2Pf/88zHXqlWrYqo//elPcasXXngh7Sqe+zOW4+zGay2S13a475dI34eq7t7b3X02hPs5o+rq8yrcz7yuPjc7+6zt7nNa1dlnfGf/L6i6+z8llgmvgkv/JxKAAQBI8QDspRDcVRAO5+QqnkG4q5NKN4NwdyfK0QRhL4dhNwJxvEOxX0NyIvdJrGHXzdCbSsE3kvDb3WdUvIJvvMJvd/+PxCP8EoABAEiTAJyMEBxNa3CyQ7BbrcGpFoTdDMNuBeJEh+J0K7eOkZuvm3iE3ngFXzdbfd0IvvEMv1317vFa+CUAAwCQRgHYLyE41iDc1UleoluD/RCEkx2G3QzEBOPkBd14BN5IQ2+qBd9kdneOtdU3lvDrZvANDr8EYAAAUjgA33///SFDcHdBONIQnKxxwcluDfZLEPZbGI5HIE73cBzP/RmP4+/H0JvI4Ov18OuV8b6hwi8BGACAFA/A0YbgdBgX7MVu0V4KwtGE4XgF4niHYj+H5ETul3gd22heZ5G+lgm+7gVfP4dfAjAAAGkQgP0egtOxNTjcIOzVVuF4h+FkhOJ0q3gfO6+GXjeDbzq1+nppvG9X4Vf/HxKAAQBIgwDspRDsxS7RBOH4h+FEBGKCsfeCbiyBN5rQS/Al/HYVfgnAAACkUQD2SwiOtUt0IlqDUzEIRxOG/RKI0z0cJ2M/x/K68GvoTVbw9WKrbyxdnuMZfgnAAACkWQBOZAj2a5doN1uD4xGEUzUMJysQ+z0ke2WfxXrsExl6Cb7eDb+d/T/jVvglAAMAkMIB+L777ktYCPZql2g/tgZ7LQgnKwx7KRBT7obdaANvrKHX68HXS92d/dTlOZLwSwAGACDFA3AqhGC/tAb7IQgnKwy7FYgJxf4Mu34JvfEY3+t28PVzq68Xwq/+TyQAAwCQ4gE4HiHYS+OCE9EanOwg7MVW4VjCsJuBmGDszbAbS+D1YuiNJvgmsrtzMlt9vTzeNzj8EoABAEiTABxNCE7WuOBkdolOxyDshTAcj0BMOE5MyHUj7LoReAm+/mv1TUb4JQADAJBGAZgu0d5sDfZSEHYrDLsRiOMdilM1ICdqn7lxfL0cer3Q1TkR3Z291uU53uGXAAwAQBoGYEKwN1uD4x2EkxmG3QrEiQzFyQ7OXnqebh07N19Pkb6Wk9na63bw9Xp353i1+roRfgnAAACkaQD2yrhggrB7QdhPYdjNQOzFYOzXcvuYuP2aiVfoTZXgS/jtPvwSgAEASOEAvGzZMkJwArtFxyMIe7FVOB5hOB6BmGCcmJAbz8Ab79BL8PV3l+dowi8BGACA1LWu4z95iqIoiqI6Kicn5wtOEQAAAID4ocUJAAAAAEAABgAAAACAAAwAAAAAAAEYAAAAAAACMAAAAAAABGAAAAAAAAjAAAAAAAAQgAEAAAAAIAADAAAAAAjAAAAAAAAQgAEAAAAAIAADAAAAAEAABgAAAACAAAwAAAAAAAEYAAAAAAACMAAAAAAABGAAAAAAAAjAAAAAAAACMAAAAAAABGAAAAAAAAjAAAAAAAAQgAEAAAAAIAADAAAAADxndUdQTNUCAAAAAIIYRVFUCtRq/vsAAADwH1rr2O88N4DXOQAAACdxYL/z3ABe5wAAAJzEgf0e2roMusfSpZb3MAAAADiJA/sdvCY4XgAAAOAkjv0OXhPgeAEAAICTOPY7eE2A4wUAAABO4tjv4DUBjhcAAAA4iWO/g9cEOF4AAADgJI79Dl4T4HgBAABwEgf2O3hNgOMFAADASRzY7+A1wfECAAAAJ3Fgv4PXBMcLAAAAnMSB/Q5eExwvAAAAcBLHfgevCXC8AAAAwEkc+x28JsDxAgAAACdxPpEdtN+z2SXgvcjxAgAAACdxqegYu4Z27PfJdk1nl4D3IscLAAAAnMSlolq71nXsd/2bzy4B70VfoRcHAAAAJ92IwOsd+30ZuwK8F32HXhwAAACcdCMCP+vY74ezK8B70XfUi+PjjuOlf+nFAQAA4AP5QSfdheyShJ5Af8WJMwjAvvVGx/G6l10BAADgDzODTrpnsksS6hR2AQjAvvXjDHpxAAAA+MrFGbtbIHUSV2XXWeySbmmym2PKKvs+VlRc8ll+z15bO/ZfwkqPWVpe8eWQ5hHPN7eM+g6HhACMpFAvjn9l0IsDAADAN47pCL066V5gVzu7pEstubl5/2geOXr9/GsWW3988q/Wk39/3/rLmo0JLT3mPStetC7++ULLDsG7qmvr1zW2tA3i8Pgaswr7E704AAAAfEStvmY5nk/DPOkutmvs2H0mf3vStBnzx0+ednfb+MnPt46b8JfmkWPeG9TUvK6xqXlDeWXVZlN9q2u2O6uib/UW598HDm76WLcb1tL6ysi28S+OHjfhgdZx7dcNHTH67PyCAs2u2uiBfdWak5P7+WVX37g50YG3uzrvkiut0rLKrZOmTWviJe1bzCqcXA12TS0pKT+laWjzL0a1tt0zYtSY54a3jFw9pGnYB3Z9Utm3arOpquqabdX9arZXVffb3vHvVvO3isq+XzUNG76ucXDT2pZRo18Z3jLqafvfOwYNGnyZ/Rhz9Dgdn70AAABIgrc6Trr/Gir0VfWrPW9wU/OjtfUD1xQU9t6a37PXTjvgWnbwtWbNnmN998z/si647JfWpVcvshYsvsO66c4HrFvufdy6/9lXwq6b734kcLsrFi4J3M/ZF863vn3KWdbMWUdbYydMsext2JGZlbXLDs6f1g8cvKq2bsCvOk4iE9X1MN9+/DXnXHT5Bq+FX2cIrus/4DNezr7F2tCJ02LXKYWFvf9QVFzydnZ29vay8vLNY8dP2HrkN46zTjr1DOuHF/3UunrhTdaiJbdbd/7xYevu+x+znv3La2GXbqNauHhp4H7Om3dJ4H5nHTnbGjW67as+RSWbs3NytvTpU/RaTk7Ozfb2HJ/hjS/6AAAAUt71HSfdv+046T68oqr6qbz8nlsGDm7afMRxx1vzLr8mEFAfffGtpAW8VW9+at3x4DPWVYtutU487Wxr+KixmzUWtrKq319z8vO/HefAcPqAxiFrvRp+TTUNH2kdeNhRF/CS9q3VHe/FB9gVrlPr+i+ysrK+qKio/OjwI7/xkYLpfY89Y61e86m15vPNCa+/v/2Rde/DK635V12z7YADD17Tu0/Rx5mZmR9k7J6boZZDBgAAEB/HdJx0/6xXQcEXk6bO2KTxrc+s/sjyeuBTKF649B479B35Va/Cwo0dQdh1PXr0ePqKhUv+6fX9oTHBYyfs+yYvad/6SQazCrtNwzoW5OblfXDS98/4+6qX30pK2A23Vv7p5Q2HHTH7OTsIa23h8zl8AAAA7tNYtK0jW8d+oO7IXg95nZW6Uat7dm5uT9dPGu0AvEUTXnl9H2hirOra+q28pH2LtaHdt7RlROsTb37w5WYvB9/gWv70n9/Pys7WvAyXcQgBAIBn5Obm3Zfh/8lMyjMzMzcnYzZjt0tjj3Pzeq6Pwz7yxfPXMSwo7L2Td6avMauwy19evfTG2i1+Cr+mbr172YsZu8eDAwAAeIa6K672cQguzsrO/uPBh89+3+/h11RGfNZRTffn70v19fUzmppH/HpIc8tT9QMGvVleUfVFMtZtTqXKycndocno+g8YvLqxadgTw0a2LuxVVNTm5c/oseMn7Hrzw/W+C8Czv/mtz3g/AwAAz51czZx11DvZ2bnv2z/Py/DPxCWaEOayvLz8T04649xP/vz2eosATABOldA7bMSoR4tLy74aMqxlp2Yp18ziN9y+LNBFPBV6OsR7KMHRc0+y+lbXBCZVC95/Dzz3qnXX8lWBsfe6XLO119T131FWXrmhqXnkXWVVtft47TN6yNDhGw48+LBdT/75774Ivg8+scqavO/+W9snT91IAAYAAJ4LwGbspX3SuFaTSOXk5GpZoZsydi9r0WpXYZK3MbBub8burpVLc3PzPigqLvlMwfehVa+l3Ak8ATg9T5jrG5uHNw5tfr2u/4Bdp593keWHMdteKu2vfacfZNU1DLSi2X+aoV1hubSsYqda3DO80ysmMFneaWf/6MOS0tJtMw85zLpm0W8CMzB7KfS+uPpd6+fXXG+Nnzh5Z3W/mq/m/WzBP/XFJAEYAAB4MgAHT0R03sVXfja2fcrbZRWV72dmZm7VGrL2dZfb9ZuM3ZOaKIxqptepHeG0IYoTxtqO27V33M9Rdp1u1+Udj7PCftxPsrKzN1dW9Vs7adqB71z884X/8vNEVwRgAnAIhaPHTXigqLhkp9aPTqXeDIkozfauVtzi0jLLjf2n+1MQ7t2naHvLqLbFGbtnYfbEZ7SC8IU/W/BB++T9Pi3s3WfnhEn7Wqf94Fxr8a2/txI9O7Rao7XGsNYGHt02bldBYe8d02Yc8tGVC3/zpfMYEIABAIDnA3BnrSvXLrlrwwWX/vK9o7918uqJU2e8PKS55dW+1TWr7RP3d/Lye67Lzs4x473CKnVf1u1KyiverupX+9qwltZXps342ivf/M733/jR/AUfXve7//0qFVt4/RaAdTJ7xcIl1gGHfD3QumYHg4SP2ywoKNS4zU2Dh7U8V1hUOjtV3nzDRk/sX98w8PNZs+dYK156l0AbYWld7+aRY6x47D990aYW5QGNTW9kJLcXTKdLqF376zs/PPHUM99RILY/h3fY781dI1vt/XHkbOvMcy6wtE6wwvGdf3w4EFjVShtOuFWYXvH8S4HbKeSqZVdB+5BZR1jDR4y0evUq2FVeUblt8r77f3LqDy5478bblv2zsy8eCMAAAMCXAZhKzwCssZINg4ZYYydMsbS2sr4IScYYVD2meiZoneDGpuZtfYqKFUpq/fzGO+CgIyaXlFdsOfOHl/K6j3LJLI3zjef+U6g79oRTrKp+tV/UNDbWev0zWl8CLLn7kU8vuuKa9xSMDzj40HUKx3Yo/rJfbd0WheRwvnAqLS3bXte/YbNuN2m//T+ZeejXP/zuGee8e9nV171/6x8f3xjJZwABGAAAEIApXwRgjYksq+gbCL5e20fnXnzFzp69Cj73awieMG3GdC33pJZ1XvMbo3ptqidCovafulbb4XFrVUNDA5/RDGkAAAAEYCrFArC6V/arrbcuuOyXnt1P5/z48m15eT3/5rc3XMPw4VUVfau3XLXoVl7vUZRaOfXaTPT+UwjuV9/wsX0I8/mMJgADAAACMJVCAVjBV92evb6vausb1mfsnozNL7KHtoxae+JpZ/Naj7JL8sSpM6xk7b+Zs462hraMfILPaAIwAAAgAFMpFIBHj5tg+aGF8sdX/mpbVk7ObX55s03Y74Dfj588jZmeoywtb5TM/acZogcNGbZr9D4Tz+czmgAMAADSIACr+2HwMkTqLqsJabqbUTWSiVN0fc3wGo/JlLy+jJIXAnBuXr4v1qHV6y4vL3+NH95omvFZ437j8bpOlxmfNe432ftP44/7FBVvyUhcV2gCMAAAQKJPrnTSqSVB1P1Qy+E0DR9pXbvkD3vCqpYh6er2p/7nj6yb7nwg7JMmXf/SqxeF/JtaJjX7azStQNoG3bd+1n2YMKznFiqEJ6MbsBcCsF9OuvWFRlZ29kY/vNHGTdj3iTknnU6YjbK077yy/6bNPNQa0z7pVwRgAjAAAEjRADxizLi9AqyCh8KwgnFwAFYw1WVqGQ4OwLpdqBYcrffrbCHuKgArrOrxIumiq8fU/TsDsHM7NKlOpAFYzy9Ua7Iuj2X9YgJw6p1gH3LEMUMKe/fZReuvv1t/na3AmhU6IzGtwARgAACARJ5cKTRq8peuuiubAKx1Y5tHjgn8rtB8892P7Am0Xzvy2EDrsYKzub5CqUKm/qZxp/Muv6bLAKzHUgDWv7qfUC27Kt2/rqMwrvvWbfTY+tlcz4Rb/asuv8Fht6sArPCt7dX96Xma7sILl96z5/lrn+lvBGBOsNun7L+Mia+ir++e+V+W1/af3t+tY9svIQATgAEAQIoFYAVRZ7g875IrA7+bVl1nAFYoNK2iCrfqKm0C7bdPOWvPfRx7wimBEKnr/u6+lXtajnX7rgKwToJN12tNhmMeq7MArHVC9Vjmcm1DcACOtAVYE+Hoeelf/a7tV7jWz41NzXtastUKrGBNAOYEu6pf7UbzOqcir4ZBQyyv7T99fg1obHqDAEwABgAAKRaAFSKdrS+33Pt4IHDqsjN/eOleAVitn87bKhjq78FjgNVSqssUIhVK1ZKq+1CA7CwAKyArqF7884WBv+k2Zrs6C8C67Ibbl4VsKY42AOv5B495VnBX4A2+fjq2ACuoHD33pMCxVLfVjueTyNLSSMvtOt4Lb7ABTSObysordxFkoyu9D8sq+npuu/TZlZffc0dG/LtBE4ABAAASeXKlcXcKM84xvSoFUIVLZwBWS41zcioFQJ0oKnQ6x+wqwJ594fxAgFagDQ6MoQKwbq/H1OWmtF16PBPIg4O3HsN5PwrzsQZg0w3bOeZX261/TYu3CezFpWVpE4D1fNXarsnFdFw1TjKSmb/dnBjrqhtu3WS/Nj4rLa941X4uVcl8g03Zf+b1Rxx3PGE2ytKa1F7df+qF0tjcMpcATAAGAAApFIBVGpurkz213KqFT622ZlywMwBrnU61/ml5GgVN/WwCrcKkxgQrrCooKljrfnVfGkOr2ypAdxaANeY3eNIp3VbBWMszKQyrdXbB4jsCwdMspaTL9Zh6bLXUhgrAzlmtnUFXl+u2pnT/ZhZYnZjreZpQbrp262Rd19W/oYJ1KgZghV996aD9kozQ29W6sZqsaNy4ieOS9QYb1tL6ir54ScYXAfoCSO9NjaF1Tszm7BXh9dLrKhn7L5zSFz2NTS13EoAJwAAAIMUCsJn5VIFToVYnpKalVyfaCp3mevOvWRwIggq35joKhJogS+FTS5mYdYP1d11P11dANUFS19fjOQOWmSArOKSalmUFc92PWpS1PSaI6XH1mHps/Wy6YjsDti5XWAoOEGassymNfzatvgrA2hfBwVn7RvdlJgRLhwCsoKUvFKJZmioRYzXLyiu3TZo2rSkZb7BBQ4Z+EMkSYG6UjoNee3ov6D2i94O+zDHviWi+mElW6XWV6P0XbukzZEhzy1MEYAIwAABIwQBMhTdbrWldU2uxc6bqVA3AZomaWJZ+infpi4vq2vp1yXiD2eH7X6GWy4r3uNng7vt6PeoYaV/oeOkLHfPllb6QMl8wqTeFXsfqweBsKdbPukxfgpnnoy+p1CtEl2kIgu5P96/rOSet0n2rJVp/i/RLEoX1RO+/SNYVr+s/8G0+ownAAACAAJy265WqVVjhQyEgmnVL/RaA1Q3UOdO2V2tI84hdg4YOPyHRb7CcnNydwePnE9ECrK7/ZtZ0Z4BUENakUgpv2i4FU31Ro9/NTOwKuwrF6tKu6+v26oqsy3Q9E67VAqrx77pcPSI0/luPp2Cs4Qy6f7VCazt0HwrAoXpydFWaST3R+y/c0vMur6j6wuuf0WZCQFM6bs59quPl/LuOpVtDGQjAAACAAEylVABWAPJqF1VnKYgNHtbyXLq8xxRwNBxBrbMav68yS3c5u0ArAJvjp7CrCd7MmHd1bTfLl6ll2FxuxuorSJlAq/DsnPVcY+NN6NWXQmoRjiZUefkzSi3qRcUlm7z++tExM19yqDREQ13kzRd0Oo76ksL8XV9qRTODPQEYAAAQgKmUD8BebqELbq0rLi37ItFvsMysrF1eGButgGNmXA8OwKaFWN2h1brrbA3UZRrPrssVdhWWnAHYOZbe2e3a3K+eux5XvQQUukKt791VZWVne3JsuUqT9xWXlH7uhwAcvHSbWurVwh/qOJo1zd3Y7wRgAABAAE5AN+Rouh4TgKMLwH55Dan1Mb9nr62JfoOVlJVvTPQYVhNYnV9MqBVWLcLB4cYZgNWiqS7Q5m/60kCllmATnrUfNYY43ACsMcLm/nVfziXEUmEMcHVN3et+DMDOL0KcLcAKxhoDbmbwJwADAIC0DMCdzRqrmWWdszUHl06iEjU5kkKvTsB1gq3ximZsYqz3291zJABnpPuXC12q6z9orWY4T0aXb7129X5Qy6veiybYaoyn6QLrDMDmdnrvaJkzdZtW12ezJJiuq/eWWe86liIGxgAAD0FJREFUnACs96BCtX7Xv8GzpndXuk0y9l84pS8UGhqHrEyFAKzWYNPqr9eKjj8twAAAgAAcInQ6J94xXR6dJ8TBrTf63YxFdJZOtEONEezs+sGlpY5MC5dZFkkTUXW1feGUtiv49+DLzHa60RWYAJxaAXjQ4KEvmqW6kjVONdLXpd4joV7jsfSs0HZEE6gUuJO5/7pb/quuYcBv/RiAdTw6+yLDrGmu1nsCMAAAIAAHrdFpZhFVq5Fp5dHvCqTqKqnLTTdI/azJcfSvWTtYt1FQVYuDWqVMC5HG1+m+Ql1f3fV0/eATNrVshdpOtSAFb5/Zft1O26pWD+eyL5o0RieJJsTr5F23V0uJ/mYmB1ILl/O+nfdBACYA1zcM/Inzixgq8iWsvLr/2tonb7EP8eF+C8Dmc8181gYHYH1hor87l7IiAAMAAAKwIwBrzJhpLdDJlTlxcrYAK8hqlllzgmVmGdWJmWm5VQuwuVwB19yPWoAVjs31NYtpqNYpdffUBD36VzOZmpZj3da0aGl7zDhEbZ8J1to2baN5DmYNX/MczIy65vG0Dbp/Xc90tdbvsc6emi4BWF8oXLFwSaCFr65hYODLko7nnrDKys7emJOT+25WTs5tcQwyjWXlldsIs9FPNKXllby2XXqvd4wpL/R6ANZnm8Z967NMpa7tzi/q1MKuz0jzd32RF+lkZQRgAACQVgFYQdWMN1TrrAmbzgCskGNOsFRmIh3nMizO+9RjdnZ9TfLT3UmzxrKZoNuroGCv+zLPxzlm0QRlndgq3Jpul+Y5qKU41HI/wc9LsyITgLufpElfVGh/6YsFHS+31h2NdGIsfXlx4RXXbqmpa/giL6/n3+znU+v2zi+tqPzMjda0dC29Vry2//T5UFnV70WvfEbT6wIAACCBAdg5PlZLpahLcXAAVqtCqJDTWQA2YTTU9UPNCqvJe4Lv32y3GevW3farC7XuR5eb8YrmOSgUq8XS+VzN2qduLgOU6gFY3c/LKvruNV7bK3XuxVfs7FVQoKWShrq58/sParzRvCeoyEs9TLy2/6bsP3Nznz59TiMAE4ABAEAKBmAFFoVUZzkDpLobq+VXQVFj9nTCasbl6nd1T1bg0fhZtf6pu7QZw9tZAFYXPHVJ1vXV2mLGsHUWgHV9dUfWfau1SN2VzVIeOnnWNqmlUdtjulAHB2AFaIVmPZfgibzMTLgaT6ztVUBX8DVjFNWSqIAc6/IhqRyAtb+0fzsbq+2Vlr28vJ7r7edV5eL+ryooLNySqBnRU3Fps+LSMssr+0+fL/bx1GsknwBMAAYAACkYgM3yGM4yodOM59TPCoKaGdW0iKoFV7+bSac06YpCsQKzaa3VGFxnoHWOPevs+p11l9VYNxN8FbhNK67ZPl2uFmpzeahxbrqd80Rb1zGPp5CrcGzCvjM06b4VhsOZsToJJ4zbI5mBN14BWME3+EsHL9apZ/9oc3Z2znNuHoDq2vprDjt6zk4CbZTHxP7MCbWUTzKqdWz71l69ep/spc9oAjAAAEAanFylYsXphPGFG25btjbZAVjd4L26pM2/dfuv6/+O/dyOd/EYFBcVl2xK5JrSqVT6EkqTYSV7/+lLueKS0vft45nNZzQBGAAAEIApb54wnj98VNuqZAdgTRCmbuh+OA4/+eWil+3nttzNg9C3X7/vVNf23xZqjV2q+9IQA006l6z9p9dun6JiLX00mc9oAjAAACAAU949YSy2a921N//+lWQGYD+93lb+be179vaud/tA2AFuUdv4STsi6ZJO7T0hlnOSukQue1TXf8DWBE18RQAGAADg5IoAHKPpmZmZH19y1XWvE4CTeyxq6hpWaYI3QnB0pcn0Ern/FH73mbTfjuqa2lv4jCYAAwAAAjDlnxPG6T169Phs6PBRf79q0dINnc2qSwCO+7EorKlv+L9t7ZN3aYZjXveRB1Kt8a2W4HjvP3V7HtjYtL1vv9p7MhI37pcADAAAwMmVe5PpaBkTLZekGZ6/e9b5W2fNnvvPqTMPXTd6n4mfDmpqXlffMPCjPkXF68Op4pLSDQMah3w0rGXU5+1Tpn0y87CjNnzz5NO3aPZrzSitJZMiGfeagBNGdYc+3w7Cf7ZrR8fj/VsRgON/LJqaR/y6ql/tDibGin5maC2rFa/9d8Pty7T80vb+Awady2c0ARgAABCAPV86MdayR4d/41vbho1o3diroEDL22zp3bvPu32Kip7Pz+/5O3sfXGbX6Rm7Z/w93K6pdrXb1RBBTe643TEd93NOXs+e/11UUnp/aXnFXwt79/4sMzNzZ13/Af+aNvPQbTpxVzA2y0V58ISRAJygY9HWPvnMouKSrVrixy8ThHlt3WatU+7m/tPnhrpZ2+/dTbW1tQfzGU0ABgAABGBPlgKl1gmeOuOQHT179dpWXtH3g8rqfv/Hfp6ndATUqiTu63y7Wu06qrq2/sa+VTVvahs1tlAtxSYME4DT8uS9eJ+J+91VUla+Y85Jpwd6KBBuI+vVoS+VFISj3X8aT3zz3Y8EgnRpWeXWkWPHLeh4z/IZTQAGAAAEYO+VujPbJ8DbBzUNe7m6ru5U+7kV+mD/5/cuKT+ssWn4M0UlpdsUhAnA6XvyPmrC/jWTph74YL/a/l8pzB1x3PGBFk4Fs/uffYVJs7opLY+kWaIbBg2xutp/+rJJv6sXht5zmlCrsHefXf0HNq7fZ9J+v/XYZwcBGAAAgJOrvevE0862mkeOeW/0+PHNfj0Q+YWF++Xk5G4lAHPyLnNO+H77QYcffeuose1rm4aP2tC3umZ7VnZ2yPHaVGSl9agrq2q2NY8cvX7MPpPeOuTI427a/+CDB/EZTQAGAAApTifUmlHV760+6jo6pK2tPAUOSXteXv6ryd6IzKysXeG2NhKAgfjRRHV+/Yxe+bc1H2TEYS1tAACAqLWNn7Rm9LgJvg3BCmkzZx1tjZ807TGOpnvqBzZuvOXexwnABGAk37Ihw1q+CDVhntfL/r/lCXv7f8UhBAAAnnHyWWf1HTai9Z0RY8YFlvzwU/DVZFf2ieHOkWP2eS4jOetzpqz9ZhzymLqVE4AJwEi6qh49ejw/fFTbP/535Us7/PC++d19Kz+rrql7SeE9wx9zMQAAgHSjpViqa+vXDWgcsu308y6yvLouqSaz0SQ3lVX9ttY1DHpjcEvLkRw9902ZftDIopLSXZpdlwAMJJ1moz4nMzPz4+YRo9+85BfXfR7OezOR9eiLb1nnXHT5O3bwfd0O7G/Z23tSBl9MAgAAH2iv7T9gcZ+i4k+LS8u+soPQxvMuuTIQPBPdTVoneGqVPvOCn+xsn7L/l1rDt6S07L2+1TVanqSFQxVfzaPG3HPAIV/vdubheAZgve7U0k8ABvYEYa0zfk9mVtY/6xsGvnP4Md9686obbt300KrXEvre0JrLP71m8bqZs456o6Jv1T/scL7B3q6ldh1F8AUAAH7VaNecvJ49/6egsPer9gnXtl6FhRsbBg1Zt/9Bh6496YxzP7nw8ms+vXbJH/YsM6JWgHBOnnSyZpYhufqm27bqfr71vf9YN2najPfr+g9Yl5ffc1NWdvbmXgWFL2ZmZ/+y46SqlkOS2JPt8sqq1ybse8DOrr78iGcA1tJWWt+VAAyEDMMz7ZrXEYi/yM7O2VTRt3pN67gJb8yee+JbP5q/4MNfXL/0C33OKrBG8vl8z4oX7c/n+3f87NqbPz73oivXHP6Nua8Pbx37eklZ+Rr7sb6yA68muLrLrnMydq+xTugFAAApSSF0esbu7m2X2/Ubu5b16NHj6ays7LU6CcsIYxkS+3qf2AF3rf3zCp28ddzPZRm7Wzcm21XFrvaEwsLevR8vK6/c8J/z5m8K1TU+UQFYLdEX/3xh4HfneHVdbq531/JV1sKl9xCAka40E367XXM6gvFvOj5fV9ifuW9H8vlsf6a/1vH5rJB7k13n23WMXWPtKmZXAwAAIJWpBf7BzMzMf4U6YU5EAJ41e4515g8vDfQamDbz0MDfdPmxJ5wSWAdal+vnvtU1BGAAAAAAQFzEPQCrS+b4ydP2XK4lYTRzuf5tGj5yr9sQgAEAAAAAvg3AWpNYLcDOv42dMCUwplH/Oi9XMCYAAwAAAAB8GYA1CZdaes2M1JqgZ+LUGYGfdbmZ3EeXZ2VnE4ABAAAAAP4MwPr5qkW3Blp31RKs0Kuwa5ZK0u9asulrRx5LCzAAAAAAID4yMzN3drdWcLQBOLjUEqzls5yPp/HB+n3FS+/uaREmAAMAAAAAXFdaUfmOWmETEYBD1dkXzg+0/qq1WC3Abq4bTAAGAAAAAPz/AFxefulhR31zR7ICsBn7qwD8u/tWunq/BGAAAAAAgFNxQe/em7QOb7ICcLyKAAwAAAAA2EufkoqDi0pKt2miKgIwAAAAACCl1dTUHFpQ2HurxuMuWHxHYHIqAjAAAAAAIFUVt45rv6mu/4DPMrOydnUEyL2KAAwAAAAASAcEYAAAAAAAAZgADAAAAAAgABOAAQAAAAAEYAIwAAAAAIAAHFU9vXrdJnt7N3LIAAAAAAAR69Gjx44/v73eFwH4ioVLPrQ3+WWOGgAAAAAg8gCcmfnir+968Cs/BOAZhx6x2t7k6zlqAAAAAIBonD9xv+lveT38PrTqNSsnJ3eDvb1jOWQAAAAAgGgUZ2ZlfXLD7cs82wr8zOqPrP4DB2v87684XAAAAACAWEzPzeu5/heLlu7wYsvvqLHtW3r2Klhmb2c2hwoAAAAAEHMIzs7O+XLfAw7etGDxHYHgmazQu+rNT62b7nzAOvaEU6xeBQXbyyurLiH8AgAAAADcVJybmz+vpLzi7cysrF0Zu9fcTXjl5OTurK1v+LBlVJsmvKrisAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEJH/B4EKwogA/asqAAAAAElFTkSuQmCC&quot; alt=&quot;Fig 1. AuditLog Architecture Figure.&quot; /&gt;&lt;/p&gt;
&lt;hr /&gt;
&lt;h2 id=&quot;what-does-it-log&quot;&gt;What does it log&lt;/h2&gt;
&lt;p&gt;Each audit log implementation has access to the following attributes. For the default text-based logger, these fields are concatenated with &lt;code class=&quot;highlighter-rouge&quot;&gt;|&lt;/code&gt; to yield the final message.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;user&lt;/code&gt;: User name(if available)&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;host&lt;/code&gt;: Host IP, where the command is being executed&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;source ip address&lt;/code&gt;: Source IP address from where the request initiated&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;source port&lt;/code&gt;: Source port number from where the request initiated&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;timestamp&lt;/code&gt;: unix time stamp&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;type&lt;/code&gt;: Type of the request (SELECT, INSERT, etc.,)&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;category&lt;/code&gt; - Category of the request (DDL, DML, etc.,)&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;keyspace&lt;/code&gt; - Keyspace(If applicable) on which request is targeted to be executed&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;scope&lt;/code&gt; - Table/Aggregate name/ function name/ trigger name etc., as applicable&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;operation&lt;/code&gt; - CQL command being executed&lt;/li&gt;
&lt;/ul&gt;
&lt;h3 id=&quot;example-of-audit-log-messages&quot;&gt;Example of Audit log messages&lt;/h3&gt;
&lt;div&gt;&lt;pre&gt;
Type: AuditLog
LogMessage: user:anonymous|host:127.0.0.1:7000|source:/127.0.0.1|port:53418|timestamp:1539978679457|type:SELECT|category:QUERY|ks:k1|scope:t1|operation:SELECT * from k1.t1 ;
Type: AuditLog
LogMessage: user:anonymous|host:127.0.0.1:7000|source:/127.0.0.1|port:53418|timestamp:1539978692456|type:SELECT|category:QUERY|ks:system|scope:peers|operation:SELECT * from system.peers limit 1;
Type: AuditLog
LogMessage: user:anonymous|host:127.0.0.1:7000|source:/127.0.0.1|port:53418|timestamp:1539980764310|type:SELECT|category:QUERY|ks:system_virtual_schema|scope:columns|operation:SELECT * from system_virtual_schema.columns ;
&lt;/pre&gt;&lt;/div&gt;
&lt;hr /&gt;
&lt;h2 id=&quot;how-to-configure&quot;&gt;How to configure&lt;/h2&gt;
&lt;p&gt;Auditlog can be configured using &lt;a href=&quot;https://github.com/apache/cassandra/blob/trunk/conf/cassandra.yaml#L1216-L1234&quot;&gt;cassandra.yaml&lt;/a&gt;. If you want to try Auditlog on one node, it can also be enabled and configured using &lt;code class=&quot;highlighter-rouge&quot;&gt;nodetool&lt;/code&gt;.&lt;/p&gt;
&lt;h4 id=&quot;cassandrayaml-configurations-for-auditlog&quot;&gt;cassandra.yaml configurations for AuditLog&lt;/h4&gt;
&lt;ul&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;enabled&lt;/code&gt;: This option enables/ disables audit log&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;logger&lt;/code&gt;: Class name of the logger/ custom logger.&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;audit_logs_dir&lt;/code&gt;: Auditlogs directory location, if not set, default to &lt;code class=&quot;highlighter-rouge&quot;&gt;cassandra.logdir.audit&lt;/code&gt; or &lt;code class=&quot;highlighter-rouge&quot;&gt;cassandra.logdir&lt;/code&gt; + /audit/&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;included_keyspaces&lt;/code&gt;: Comma separated list of keyspaces to be included in audit log, default - includes all keyspaces&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;excluded_keyspaces&lt;/code&gt;: Comma separated list of keyspaces to be excluded from audit log, default - excludes no keyspace&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;included_categories&lt;/code&gt;: Comma separated list of Audit Log Categories to be included in audit log, default - includes all categories&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;excluded_categories&lt;/code&gt;: Comma separated list of Audit Log Categories to be excluded from audit log, default - excludes no category&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;included_users&lt;/code&gt;: Comma separated list of users to be included in audit log, default - includes all users&lt;/li&gt;
&lt;li&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;excluded_users&lt;/code&gt;: Comma separated list of users to be excluded from audit log, default - excludes no user&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Note: BinAuditLogger configurations can be tuned using cassandra.yaml properties as well.&lt;/p&gt;
&lt;p&gt;List of available categories are: QUERY, DML, DDL, DCL, OTHER, AUTH, ERROR, PREPARE&lt;/p&gt;
&lt;h4 id=&quot;nodetool-command-to-enable-auditlog&quot;&gt;NodeTool command to enable AuditLog&lt;/h4&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;enableauditlog&lt;/code&gt;: Enables AuditLog with yaml defaults. yaml configurations can be overridden using options via nodetool command.&lt;/p&gt;
&lt;div class=&quot;highlighter-rouge&quot;&gt;&lt;pre class=&quot;highlight&quot;&gt;&lt;code&gt;nodetool enableauditlog
&lt;/code&gt;&lt;/pre&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Options:&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;--excluded-categories&lt;/code&gt;
Comma separated list of Audit Log Categories to be excluded for
audit log. If not set the value from cassandra.yaml will be used&lt;/p&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;--excluded-keyspaces&lt;/code&gt;
Comma separated list of keyspaces to be excluded for audit log. If
not set the value from cassandra.yaml will be used&lt;/p&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;--excluded-users&lt;/code&gt;
Comma separated list of users to be excluded for audit log. If not
set the value from cassandra.yaml will be used&lt;/p&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;--included-categories&lt;/code&gt;
Comma separated list of Audit Log Categories to be included for
audit log. If not set the value from cassandra.yaml will be used&lt;/p&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;--included-keyspaces&lt;/code&gt;
Comma separated list of keyspaces to be included for audit log. If
not set the value from cassandra.yaml will be used&lt;/p&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;--included-users&lt;/code&gt;
Comma separated list of users to be included for audit log. If not
set the value from cassandra.yaml will be used&lt;/p&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;--logger&lt;/code&gt;
Logger name to be used for AuditLogging. Default BinAuditLogger. If
not set the value from cassandra.yaml will be used&lt;/p&gt;
&lt;h4 id=&quot;nodetool-command-to-disable-auditlog&quot;&gt;NodeTool command to disable AuditLog&lt;/h4&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;disableauditlog&lt;/code&gt;: Disables AuditLog.&lt;/p&gt;
&lt;div class=&quot;highlighter-rouge&quot;&gt;&lt;pre class=&quot;highlight&quot;&gt;&lt;code&gt;nodetool disableuditlog
&lt;/code&gt;&lt;/pre&gt;
&lt;/div&gt;
&lt;h4 id=&quot;nodetool-command-to-reload-auditlog-filters&quot;&gt;NodeTool command to reload AuditLog filters&lt;/h4&gt;
&lt;p&gt;&lt;code class=&quot;highlighter-rouge&quot;&gt;enableauditlog&lt;/code&gt;: NodeTool enableauditlog command can be used to reload auditlog filters when called with default or previous &lt;code class=&quot;highlighter-rouge&quot;&gt;loggername&lt;/code&gt; and updated filters&lt;/p&gt;
&lt;div class=&quot;highlighter-rouge&quot;&gt;&lt;pre class=&quot;highlight&quot;&gt;&lt;code&gt;nodetool enableauditlog --loggername &amp;lt;Default/ existing loggerName&amp;gt; --included-keyspaces &amp;lt;New Filter values&amp;gt;
&lt;/code&gt;&lt;/pre&gt;
&lt;/div&gt;
&lt;hr /&gt;
&lt;h2 id=&quot;conclusion&quot;&gt;Conclusion&lt;/h2&gt;
&lt;p&gt;Now that Apache Cassandra ships with audit logging out of the box, users
can easily capture data change events to a persistent record indicating
what happened, when it happened, and where the event originated. This
type of information remains critical to modern enterprises operating in
a diverse regulatory environment. While audit logging represents one of
many steps forward in the 4.0 release, we believe that it will uniquely
enable enterprises to use the database in ways they could not
previously.&lt;/p&gt;</content><author><name>the Apache Cassandra Community</name></author><summary type="html">Database audit logging is an industry standard tool for enterprises to capture critical data change events including what data changed and who triggered the event. These captured records can then be reviewed later to ensure compliance with regulatory, security and operational policies.</summary></entry><entry><title type="html">Finding Bugs in Cassandra’s Internals with Property-based Testing</title><link href="http://cassandra.apache.org/blog/2018/10/17/finding_bugs_with_property_based_testing.html" rel="alternate" type="text/html" title="Finding Bugs in Cassandra's Internals with Property-based Testing" /><published>2018-10-17T07:00:00+00:00</published><updated>2018-10-17T07:00:00+00:00</updated><id>http://cassandra.apache.org/blog/2018/10/17/finding_bugs_with_property_based_testing</id><content type="html" xml:base="http://cassandra.apache.org/blog/2018/10/17/finding_bugs_with_property_based_testing.html">&lt;p&gt;As of September 1st, the Apache Cassandra community has shifted the focus of Cassandra 4.0 development from new feature work to testing, validation, and hardening, with the goal of releasing a stable 4.0 that every Cassandra user, from small deployments to large corporations, can deploy with confidence. There are several projects and methodologies that the community is undertaking to this end. One of these is the adoption of property-based testing, which was &lt;a href=&quot;http://cassandra.apache.org/blog/2018/08/21/testing_apache_cassandra.html&quot;&gt;previously introduced here&lt;/a&gt;. This post will take a look at a specific use of this approach and how it found a bug in a new feature meant to ensure data integrity between the client and Cassandra.&lt;/p&gt;
&lt;h4 id=&quot;detecting-corruption-is-a-property&quot;&gt;Detecting Corruption is a Property&lt;/h4&gt;
&lt;p&gt;In this post, we demonstrate property-based testing in Cassandra through the integration of the &lt;a href=&quot;https://github.com/ncredinburgh/QuickTheories&quot;&gt;QuickTheories&lt;/a&gt; library introduced as part of the work done for &lt;a href=&quot;https://issues.apache.org/jira/browse/CASSANDRA-13304&quot;&gt;CASSANDRA-13304&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This ticket modifies the framing of Cassandra’s native client protocol to include checksums in addition to the existing, optional compression. Clients can opt-in to this new feature to retain data integrity across the many hops between themselves and Cassandra. This is meant to address cases where hardware and protocol level checksums fail (due to underlying hardware issues) — a case that has been seen in production. A description of the protocol changes can be found in the ticket but for the purposes of this discussion the salient part is that two checksums are added: one that covers the length(s) of the data (if compressed there are two lengths), and one for the data itself. Before merging this feature, property-based testing using QuickTheories was used to uncover a bug in the calculation of the checksum over the lengths. This bug could have led to silent corruption at worst or unexpected errors during deserialization at best.&lt;/p&gt;
&lt;p&gt;The test used to find this bug is shown below. This example tests the property that when a frame is corrupted, that corruption should be caught by checksum comparison. The test is wrapped inside of a standard JUnit test case but, once called by JUnit, execution is handed over to QuickTheories to generate and execute hundreds of examples. These examples are dictated by the types of input that should be generated (the arguments to &lt;code class=&quot;highlighter-rouge&quot;&gt;forAll&lt;/code&gt;). The execution of each individual example is done by &lt;code class=&quot;highlighter-rouge&quot;&gt;checkAssert&lt;/code&gt; and its argument, the &lt;code class=&quot;highlighter-rouge&quot;&gt;roundTripWithCorruption&lt;/code&gt; function.&lt;/p&gt;
&lt;div&gt;&lt;div&gt;&lt;pre&gt;
@Test
public void corruptionCausesFailure()
{
qt().withExamples(500)
.forAll(inputWithCorruptablePosition(),
integers().between(0, Byte.MAX_VALUE).map(Integer::byteValue),
compressors(),
checksumTypes())
.checkAssert(this::roundTripWithCorruption);
}
&lt;/pre&gt;&lt;/div&gt;&lt;/div&gt;
&lt;p&gt;The &lt;code class=&quot;highlighter-rouge&quot;&gt;roundTripWithCorruption&lt;/code&gt; function is a generalization of a unit test that worked similarly but for a single case. It is given an input to transform and a position in the transformed output to insert corruption, as well as what byte to write to the corrupted position. The additional arguments (the compressor and checksum type) are used to ensure coverage of Cassandra’s various compression and checksumming implementations.&lt;/p&gt;
&lt;div&gt;&lt;div&gt;&lt;pre&gt;
private void roundTripWithCorruption(Pair&amp;lt;String, Integer&amp;gt; inputAndCorruptablePosition,
byte corruptionValue,
Compressor compressor,
ChecksumType checksum) {
String input = inputAndCorruptablePosition.left;
ByteBuf expectedBuf = Unpooled.wrappedBuffer(input.getBytes());
int byteToCorrupt = inputAndCorruptablePosition.right;
ChecksummingTransformer transformer = new ChecksummingTransformer(checksum, DEFAULT_BLOCK_SIZE, compressor);
ByteBuf outbound = transformer.transformOutbound(expectedBuf);
// make sure we're actually expecting to produce some corruption
if (outbound.getByte(byteToCorrupt) == corruptionValue)
return;
if (byteToCorrupt &amp;gt;= outbound.writerIndex())
return;
try {
int oldIndex = outbound.writerIndex();
outbound.writerIndex(byteToCorrupt);
outbound.writeByte(corruptionValue);
outbound.writerIndex(oldIndex);
ByteBuf inbound = transformer.transformInbound(outbound, FLAGS);
// verify that the content was actually corrupted
expectedBuf.readerIndex(0);
Assert.assertEquals(expectedBuf, inbound);
} catch(ProtocolException e) {
return;
}
}
&lt;/pre&gt;&lt;/div&gt;&lt;/div&gt;
&lt;p&gt;The remaining piece is how those arguments are generated — the arguments to &lt;code class=&quot;highlighter-rouge&quot;&gt;forAll&lt;/code&gt; mentioned above. Each argument is a function that returns an input generator. For each example, an input is pulled from each generator and passed to &lt;code class=&quot;highlighter-rouge&quot;&gt;roundTripWithCorruption&lt;/code&gt;. The &lt;code class=&quot;highlighter-rouge&quot;&gt;compressors()&lt;/code&gt; and &lt;code class=&quot;highlighter-rouge&quot;&gt;checksums()&lt;/code&gt; generators aren’t copied here. They can be found in the &lt;a href=&quot;https://github.com/apache/cassandra/blob/65fb17a88bd096b1e952ccca31ad709759644a1b/test/unit/org/apache/cassandra/transport/frame/checksum/ChecksummingTransformerTest.java#L209-L217&quot;&gt;source&lt;/a&gt; and are based on built-in generator methods, provided by QuickTheories, that select a value from a list of values. The second argument, &lt;code class=&quot;highlighter-rouge&quot;&gt;integers().between(0, Byte.MAX_VALUE).map(Integer::byteValue)&lt;/code&gt;, generates non-negative numbers that fit into a single byte. These numbers will be passed as the &lt;code class=&quot;highlighter-rouge&quot;&gt;corruptionValue&lt;/code&gt; argument.&lt;/p&gt;
&lt;p&gt;The &lt;code class=&quot;highlighter-rouge&quot;&gt;inputWithCorruptiblePosition&lt;/code&gt; generator, copied below, generates strings to use as input to the transformation function and a position within the output byte stream to corrupt. Because compression prevents knowledge of the output size of the frame, the generator tries to choose a somewhat reasonable position to corrupt by limiting the choice to the size of the generated string (it’s uncommon for compression to generate a larger string and the implementation discards the compressed value if it does). It also avoids corrupting the first two bytes of the stream which are not covered by a checksum and therefore can be corrupted without being caught. The function above ensures that corruption is actually introduced and that corrupting a position larger than the size of the output does not occur.&lt;/p&gt;
&lt;div&gt;&lt;div&gt;&lt;pre&gt;
private Gen&amp;lt;Pair&amp;lt;String, Integer&amp;gt;&amp;gt; inputWithCorruptablePosition()
{
return inputs().flatMap(s -&amp;gt; integers().between(2, s.length() + 2)
.map(i -&amp;gt; Pair.create(s, i)));
}
&lt;/pre&gt;&lt;/div&gt;&lt;/div&gt;
&lt;p&gt;With all those pieces in place, if the test were run before the bug were fixed, it would fail with the following output.&lt;/p&gt;
&lt;div&gt;&lt;div&gt;&lt;pre&gt;
java.lang.AssertionError: Property falsified after 2 example(s)
Smallest found falsifying value(s) :-
{(c,3), 0, null, Adler32}
Cause was :-
java.lang.IndexOutOfBoundsException: readerIndex(10) + length(16711681) exceeds writerIndex(15): UnpooledHeapByteBuf(ridx: 10, widx: 15, cap: 54/54)
at io.netty.buffer.AbstractByteBuf.checkReadableBytes0(AbstractByteBuf.java:1401)
at io.netty.buffer.AbstractByteBuf.checkReadableBytes(AbstractByteBuf.java:1388)
at io.netty.buffer.AbstractByteBuf.readBytes(AbstractByteBuf.java:870)
at org.apache.cassandra.transport.frame.checksum.ChecksummingTransformer.transformInbound(ChecksummingTransformer.java:289)
at org.apache.cassandra.transport.frame.checksum.ChecksummingTransformerTest.roundTripWithCorruption(ChecksummingTransformerTest.java:106)
...
Other found falsifying value(s) :-
{(c,3), 0, null, CRC32}
{(c,3), 1, null, CRC32}
{(c,3), 9, null, CRC32}
{(c,3), 11, null, CRC32}
{(c,3), 36, null, CRC32}
{(c,3), 50, null, CRC32}
{(c,3), 74, null, CRC32}
{(c,3), 99, null, CRC32}
Seed was 179207634899674
&lt;/pre&gt;&lt;/div&gt;&lt;/div&gt;
&lt;p&gt;The output shows more than a single failing example. This is because QuickTheories, like most property-based testing libraries, comes with a shrinker, which performs the task of taking a failure and minimizing its inputs. This aids in debugging because there are multiple failing examples to look at often removing noise in the process. Additionally, a seed value is provided so the same series of tests and failures can be generated again — another useful feature when debugging. In this case, the library generated an example that contains a single byte of input, which will corrupt the fourth byte in the output stream by setting it to zero, using no compression, and using Adler32 for checksumming. It can be seen from the other failing examples that using CRC32 also fails. This is due to improper calculation of the checksum, regardless of the algorithm. In particular, the checksum was only calculated over the least significant byte of each length rather than all eight bytes. By corrupting the fourth byte of the output stream (the first length’s second-most significant byte not covered by the calculation), an invalid length is read and later used.&lt;/p&gt;
&lt;h4 id=&quot;where-to-find-more&quot;&gt;Where to Find More&lt;/h4&gt;
&lt;p&gt;Property-based testing is a broad topic, much of which is not covered by this post. In addition to Cassandra, it has been used successfully in several places including &lt;a href=&quot;https://ieeexplore.ieee.org/document/7107466/&quot;&gt;car&lt;/a&gt; &lt;a href=&quot;https://arxiv.org/pdf/1703.06574.pdf&quot;&gt;operating
systems&lt;/a&gt; and &lt;a href=&quot;https://youtu.be/hXnS_Xjwk2Y?t=1023&quot;&gt;suppliers’ products&lt;/a&gt;, &lt;a href=&quot;https://dl.acm.org/citation.cfm?id=2034662&quot;&gt;GNOME Glib&lt;/a&gt;, &lt;a href=&quot;https://github.com/WesleyAC/raft/tree/master/src&quot;&gt;distributed consensus&lt;/a&gt;, and other &lt;a href=&quot;https://www.youtube.com/watch?v=x9mW54GJpG0&quot;&gt;distributed&lt;/a&gt; &lt;a href=&quot;https://youtu.be/hXnS_Xjwk2Y?t=1382&quot;&gt;databases&lt;/a&gt;. It can also be combined with other approaches such as fault-injection and memory leak detection. Stateful models can also be built to generate a series of commands instead of running each example on one generated set of inputs. Our goal is to evangelize this approach within the Cassandra developer community and encourage more testing of this kind as part of our work to deliver the most stable major release of Cassandra yet.&lt;/p&gt;</content><author><name>the Apache Cassandra Community</name></author><summary type="html">As of September 1st, the Apache Cassandra community has shifted the focus of Cassandra 4.0 development from new feature work to testing, validation, and hardening, with the goal of releasing a stable 4.0 that every Cassandra user, from small deployments to large corporations, can deploy with confidence. There are several projects and methodologies that the community is undertaking to this end. One of these is the adoption of property-based testing, which was previously introduced here. This post will take a look at a specific use of this approach and how it found a bug in a new feature meant to ensure data integrity between the client and Cassandra.</summary></entry><entry><title type="html">Testing Apache Cassandra 4.0</title><link href="http://cassandra.apache.org/blog/2018/08/21/testing_apache_cassandra.html" rel="alternate" type="text/html" title="Testing Apache Cassandra 4.0" /><published>2018-08-21T03:00:00+00:00</published><updated>2018-08-21T03:00:00+00:00</updated><id>http://cassandra.apache.org/blog/2018/08/21/testing_apache_cassandra</id><content type="html" xml:base="http://cassandra.apache.org/blog/2018/08/21/testing_apache_cassandra.html">&lt;p&gt;With the goal of ensuring reliability and stability in Apache Cassandra 4.0, the project’s committers have voted to freeze new features on September 1 to concentrate on testing and validation before cutting a stable beta. Towards that goal, the community is investing in methodologies that can be performed at scale to exercise edge cases in the largest Cassandra clusters. The result, we hope, is to make Apache Cassandra 4.0 the best-tested and most reliable major release right out of the gate.&lt;/p&gt;
&lt;p&gt;In the interests of communication (and hopefully more participation), here’s a look at some of the approaches being used to test Apache Cassandra 4.0:&lt;/p&gt;
&lt;hr /&gt;
&lt;h4 id=&quot;replay-testing&quot;&gt;Replay Testing&lt;/h4&gt;
&lt;h5 id=&quot;workload-recording-log-replay-and-comparison&quot;&gt;Workload Recording, Log Replay, and Comparison&lt;/h5&gt;
&lt;p&gt;Replay testing allows for side-by-side comparison of a workload using two versions of the same database. It is a black-box technique that answers the question, “did anything change that we didn’t expect?”&lt;/p&gt;
&lt;p&gt;Replay testing is simple in concept: record a workload, then re-issue it against two clusters – one running a stable release and the second running a candidate build. Replay testing a stateful distributed system is more challenging. For a subset of workloads, we can achieve determinism in testing by grouping writes by CQL partition and ordering them via client-supplied timestamps. This also allows us to achieve parallelism, as recorded workloads can be distributed by partition across an arbitrarily-large fleet of writers. Though linearizing updates within a partition and comparing differences does not allow for validation of all possible workloads (e.g., CAS queries), this subset is very useful.&lt;/p&gt;
&lt;p&gt;The suite of Full Query Logging (“FQL”) tools in Apache Cassandra enable workload recording. &lt;a href=&quot;https://issues.apache.org/jira/browse/CASSANDRA-14618&quot;&gt;CASSANDRA-14618&lt;/a&gt; and &lt;a href=&quot;https://issues.apache.org/jira/browse/CASSANDRA-14619&quot;&gt;CASSANDRA-14619&lt;/a&gt; will add fqltool replay and fqltool compare, enabling log replay and comparison. Standard tools in the Apache ecosystem such as &lt;a href=&quot;https://spark.apache.org&quot;&gt;Apache Spark&lt;/a&gt; and &lt;a href=&quot;https://mesos.apache.org&quot;&gt;Apache Mesos&lt;/a&gt; can also make parallelizing replay and comparison across large clusters of machines straightforward.&lt;/p&gt;
&lt;hr /&gt;
&lt;h4 id=&quot;fuzz-testing-and-property-based-testing&quot;&gt;Fuzz Testing and Property-Based Testing&lt;/h4&gt;
&lt;h5 id=&quot;dynamic-test-generation-and-fuzzing&quot;&gt;Dynamic Test Generation and Fuzzing&lt;/h5&gt;
&lt;p&gt;Fuzz testing dynamically generates input to be passed through a function for validation. We can make fuzz testing smarter in stateful systems like Apache Cassandra to assert that persisted data conforms to the database’s contracts: acknowledged writes are not lost, deleted data is not resurrected, and consistency levels are respected. Fuzz testing of storage systems to validate these properties requires maintaining a record of responses received from the system; the development of a model representing valid legal states of data within the database; and a validation pass to assert that responses reflect valid states according to that model.&lt;/p&gt;
&lt;p&gt;Property-based testing combines fuzz testing and assertions to explore a state space using randomly-generated input. These tests provide dynamic input to the system and assert that its fundamental properties are not violated. These properties can range from generic (e.g., “I can write data and read it back”) to specific (“range tombstone bounds synthesized during short-read-protection reads are properly closed”); and from local to distributed (e.g., “replacing every single node in a cluster results in an identical database”). To simplify debugging, property-based testing libraries like &lt;a href=&quot;https://github.com/ncredinburgh/QuickTheories&quot;&gt;QuickTheories&lt;/a&gt; also provide a “shrinker,” which attempts to generate the simplest possible failing case after detecting input or a sequence of actions that triggers a failure.&lt;/p&gt;
&lt;p&gt;Unlike model checkers, property-based tests don’t exhaust the state space – but explore it until a threshold of examples is reached. This allows for the computation to be distributed across many machines to gain confidence in code and infrastructure that scales with the amount of computation applied to test it.&lt;/p&gt;
&lt;hr /&gt;
&lt;h4 id=&quot;distributed-tests-and-fault-injection-testing&quot;&gt;Distributed Tests and Fault-Injection Testing&lt;/h4&gt;
&lt;h5 id=&quot;validating-behavior-under-fault-scenarios&quot;&gt;Validating Behavior Under Fault Scenarios&lt;/h5&gt;
&lt;p&gt;All of the above techniques can be combined with fault injection testing to validate that the system maintains availability where expected in fault scenarios, that fundamental properties hold, and that reads and writes conform to the system’s contracts. By asserting series of invariants under fault scenarios using different techniques, we gain the ability to exercise edge cases in the system that may reveal unexpected failures in extreme scenarios. Injected faults can take many forms – network partitions, process pauses, disk failures, and more.&lt;/p&gt;
&lt;hr /&gt;
&lt;h4 id=&quot;upgrade-testing&quot;&gt;Upgrade Testing&lt;/h4&gt;
&lt;h5 id=&quot;ensuring-a-safe-upgrade-path&quot;&gt;Ensuring a Safe Upgrade Path&lt;/h5&gt;
&lt;p&gt;Finally, it’s not enough to test one version of the database. Upgrade testing allows us to validate the upgrade path between major versions, ensuring that a rolling upgrade can be completed successfully, and that contents of the resulting upgraded database is identical to the original. To perform upgrade tests, we begin by snapshotting a cluster and cloning it twice, resulting in two identical clusters. One of the clusters is then upgraded. Finally, we perform a row-by-row scan and comparison of all data in each partition to assert that all rows read are identical, logging any deltas for investigation. Like fault injection tests, upgrade tests can also be thought of as an operational scenario all other types of tests can be parameterized against.&lt;/p&gt;
&lt;hr /&gt;
&lt;h4 id=&quot;wrapping-up&quot;&gt;Wrapping Up&lt;/h4&gt;
&lt;p&gt;The Apache Cassandra developer community is working hard to deliver Cassandra 4.0 as the most stable major release to date, bringing a variety of methodologies to bear on the problem. We invite you to join us in the effort, deploying these techniques within your infrastructure and testing the release on your workloads. Learn more about how to get involved &lt;a href=&quot;http://cassandra.apache.org/community/&quot;&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The more that join, the better the release we’ll ship together.&lt;/p&gt;</content><author><name>the Apache Cassandra Community</name></author><summary type="html">With the goal of ensuring reliability and stability in Apache Cassandra 4.0, the project’s committers have voted to freeze new features on September 1 to concentrate on testing and validation before cutting a stable beta. Towards that goal, the community is investing in methodologies that can be performed at scale to exercise edge cases in the largest Cassandra clusters. The result, we hope, is to make Apache Cassandra 4.0 the best-tested and most reliable major release right out of the gate.</summary></entry><entry><title type="html">Hardware-bound Zero Copy Streaming in Apache Cassandra 4.0</title><link href="http://cassandra.apache.org/blog/2018/08/07/faster_streaming_in_cassandra.html" rel="alternate" type="text/html" title="Hardware-bound Zero Copy Streaming in Apache Cassandra 4.0" /><published>2018-08-07T19:00:00+00:00</published><updated>2018-08-07T19:00:00+00:00</updated><id>http://cassandra.apache.org/blog/2018/08/07/faster_streaming_in_cassandra</id><content type="html" xml:base="http://cassandra.apache.org/blog/2018/08/07/faster_streaming_in_cassandra.html">&lt;p&gt;Streaming in Apache Cassandra powers host replacement, range movements, and cluster expansions. Streaming plays a crucial role in the cluster and as such its performance is key to not only the speed of the operations its used in but the cluster’s health generally. In Apache Cassandra 4.0, we have introduced an improved streaming implementation that reduces GC pressure and increases throughput several folds and are now limited, in some cases, only by the disk / network IO (See: &lt;a href=&quot;https://issues.apache.org/jira/browse/CASSANDRA-14556&quot;&gt;CASSANDRA-14556&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;&lt;img src=&quot;data:image/png;base64,iVBORw0KGgoAAAANSUhEUgAAAU8AAAFRCAYAAAD0ES8ZAAAAAXNSR0IArs4c6QAAACBjSFJNAAB6JgAAgIQAAPoAAACA6AAAdTAAAOpgAAA6mAAAF3CculE8AAAACXBIWXMAAA7EAAAOxAGVKw4bAAACC2lUWHRYTUw6Y29tLmFkb2JlLnhtcAAAAAAAPHg6eG1wbWV0YSB4bWxuczp4PSJhZG9iZTpuczptZXRhLyIgeDp4bXB0az0iWE1QIENvcmUgNS40LjAiPgogICA8cmRmOlJERiB4bWxuczpyZGY9Imh0dHA6Ly93d3cudzMub3JnLzE5OTkvMDIvMjItcmRmLXN5bnRheC1ucyMiPgogICAgICA8cmRmOkRlc2NyaXB0aW9uIHJkZjphYm91dD0iIgogICAgICAgICAgICB4bWxuczp0aWZmPSJodHRwOi8vbnMuYWRvYmUuY29tL3RpZmYvMS4wLyI+CiAgICAgICAgIDx0aWZmOlJlc29sdXRpb25Vbml0PjI8L3RpZmY6UmVzb2x1dGlvblVuaXQ+CiAgICAgICAgIDx0aWZmOkNvbXByZXNzaW9uPjU8L3RpZmY6Q29tcHJlc3Npb24+CiAgICAgICAgIDx0aWZmOk9yaWVudGF0aW9uPjE8L3RpZmY6T3JpZW50YXRpb24+CiAgICAgICAgIDx0aWZmOlBob3RvbWV0cmljSW50ZXJwcmV0YXRpb24+MjwvdGlmZjpQaG90b21ldHJpY0ludGVycHJldGF0aW9uPgogICAgICA8L3JkZjpEZXNjcmlwdGlvbj4KICAgPC9yZGY6UkRGPgo8L3g6eG1wbWV0YT4KhtKuJAAAQABJREFUeAHsfQmAFMXVf3X3XHtf3JeIoMDG2xzeHBovbtxNYmLMCcmnQeAfBbwYFAQkgqCJwSPRL1Hz7SqnRzy4vI0ajcoqBgURudn7mLP7/3vVXcsAe8zMzs727FbB7PR0V1VX/br716/ee/VKYTJJBNoLAcNQWHGpyqsvLQ4fe5rcZatzVb/Wl4X1frqm9VQMPZsxJUNRjEydqVn4remM1aiKUqvqeq0D3wFDOWAY2m63y79n/02TDhxbJ6NzluKcRUU6Q0XHHZc7JAIJQsCRoHpkNRKBIwiUlGisrMwAeYH7GCfNvEUv5zAWGMDU8LeYrow0mHE687M83TDymKp0U9xpjKngWUMH/4FC8U0JxIkfCoqoLIADhr+BMT1U4Q9rlbmL1lUYhrENebboivF+OOz8qlZRDopzojKFeedpzDsXbZBEygGVfxKGAO5MmSQCCUBASHzFRyTMwSuezz5cG7oIDDjOMPRzQGanK640UxIlUtRBlHoI9Mr5lf4QbdJXU/elkCIdTNXAqhpTNHxTEfofCqAu4zNU/h/w5LNBLfxSbaRkKgjd6zVZOQFdllV0bQSaukm7NiKy9zEiAKIrwTA5gjR7LVw/IqCyMbqhTwAPnqR40pgRBkkGA1Q3htOgO6JCzpP8L21Fdy+aZU3GPCJNUlmVOZxMcbiYEfCBSPV94OcXsH9tX5/vha3eYn5yRiQqh/SARaa2IhDdDdvWs8jynRMBIiKLNPstfTOtPnjgKt1Qf4vOjlTc6YoR9EOqBGkyhj+GiuE3UV873nM01ueqAi6dKk43l0gVPfQhzvtoyAg/VXPLpMP8YvC2Qy96hID5bvlHIhAtAu14I0fbBJkv5RAoAmmWmAaZQm+Ja0+a51cYMk+B1Hc66S05aRoGsSYIE//alTCbQE9ItqaUC4nUrSjUroBvBwj8cbfqenD/TZeZxqaIF0ATNcldEoFmEZDk2Sw08kATCGCIXtI4RM9fuPZyDNpvZ073eTQMB2mSPpEG5KTXtMe9RURKJEp/HU5N0Rwg0YYvNEOZf3jOuMewlzGvV2Vz55KBi3LJJBGICgF73OBRNVVm6lAEvBh2ezm5GN0Wv3BKmAXuMphapGhOkKZPDH/JgmPjxIf1BidRWO+NkP9VaBPmVs4Zu5k3WkqhNr529muaJE/7XRP7tcjrdUA648rLvEXrpsLaswg6zVzDV09tJSu5zUnzWEg5iTLF7VENv485VWVBYUOWd7N3ZIgblCKMX8eWlL8lAgIBSZ4CCfndNAKWNJaz8Nk8eFsuV5zOaw1yLdJhCVJAmsnWZzbdynj3oiOKxlxuuKD6XtVU7TeHZ435VA7j44Wza5WT5Nm1rncMvSUHc9CjV9HzFq2/AH6aj8KR/WQ4qZNek5Lpr2lup/JfrhNVXB4NUmiloqk3VMwa+wTvEOlCpV9oKl/bdm27JM92hTdFKyeHd0owoOQtWvtDbP0FRqE0I+CHtAlJTRznmTrNnxDTHHzGHWxMt1bcPOZu3jPyLGhiammn6bXsSNwISPKMG7pOWpCI0bI6Q+K8HhLnA3xGDw3TYa/upL02u6VAf2swDVIoLPL+FZVzxt3ID0gC7dSXPd7OSfKMF7nOWO4o4ly3wNAct3And0zXgRjaWYbprV05UksoUFEoLOD/R59Bg67bWlwYYKa3gVBZtFaHPN4FEOgqD0QXuJRt7GLEUDx/0dq7mCsNxBmEYZ1bprvSfcL7avjrdcPl+eHeL778u5frPjFziQhUJomAhYC8GeStQAgobN48PlwvWLx+mqE5bwN50PxzMqZ0xXsEIzJMJfXXhw23u2i5+8wH+W0C41kn1ffy7sk/sSEgh+2x4dU5c1s6vfzF63+OEHF/4Z3kEmeXJM7Ia0wvDwNDeJX5/Qsr5oy9hZMneJX2R2aU210Pga4oVXS9q9xSj72bHGRNzlm8fjSiID1MApc5VO/yxEmomSxJMURdrjkFi9fewI1pNEVVpi6PgJQ8u/ItYEmcve5+vrtPCb0NB/hBRjBIEZA6t1U99muOMHqqisDLPs2hjCq/aexbciZS7CB2thLyDdrZrmjU/YFLkuW/6FOCKzE0BXEGJHE2jR8mwuthxenyhMPGozTbiofiI0OSTF0WAXnxu+ql95pD0ryFa+fAr3Gi4a8jNxwpcTZ/P2gs6A+p7vRhqqKv4NkoEpNMXRYBSZ5d8dLTcB2W4x5L1p6G7t8GiRNfUoPT6q2AufyGrw7xnrWf5C9aV2zpP1MsKEqrvZQZokRAkmeUQHWibI3D9WCY3c3caenmQkLSQNTqNeZBUCBswqgGMX1+1t2rCuTwvVXUOm0GSZ6d9tI20zHLUoypl9cwh+sqBPogSpDSUzNwHb8bM61CwRBC8g1xKM5Z/PjW4VJsPx6oTr9HXvROf4kjOmhNvxyEZYArWP2bhuYazkIU/b3LTL2MAKNNm4gDjQWPdaMOSyadWzF7wsdMzn9vE6CpWFhKnql41eJtcylWuUSqZA2/Zi4PiDNA8SzlPRA7nhi3I6ipJy0DYvt0XrwQ69TL1KUQkJJnV7ncltSZ612dyzzavxB+bQjIU0qd8V9/8KaK6HV6LYJQncelTxk8JH40U7CklDpS8KLF1WRL6jTStWIsySuJMy4QjypExBmGf2wm3ktTzCPzjsogf3RuBKTk2UmvrwG5qLS0iL8cS1kR/OGLw2zF8+7c+uCbIM+zEK8yBdcest3F4rpPDOGrnEw55+Cc8dv5Eh4y+rztLlR7NEg6RbcHqh1Up9fL1K3Di5QinF9RSkGO9KFUyv/mNxgX6Yp2FpYIJgs7Zs3w3fJP/AhY0md6Tsjvm4xqFrPh0vIeP5ypVVKSZ2pdr6Zaq3gNrzK8dKtSXMwJk1NliVGk7dkQPstQ9UsM3bFj5iVP/0MxQuPhYsOw6iXNJpLuSU2hGes+6wWE6H1jUHQx9/vEuwvb8tUUK5Ypll8O21PsgonmFpUUaSRhmoRp7l3yn2sznJV1ZxshYxL2XOl0q0M86U5WW1n78PRl3ukF5+3+THe4+huhIJGn1HebsLX1L6R4HooqqIaUi8pvG/e2DBrSVkhTo7yUPFPjOvFWCj1mWVmp4YWUSYNxinKedeEn31NZeCI7XFOUluk8IeALs2BQ/yAYMO4IBJRXZ456dgtb+NsfYVmN/kY4REN2DDdTqOP2biqwNMKKJ8Ops/rvo6lvs7Iiia69r1lCWifJMyEwtm8lpMscDl1mpB5z6aZxgxXmGMP0D3/h8jhOpUUt/b7Qew214YfgOrOu+rUzykCsJGHylK+ELzZcGYwhA3bI627hkpgvQzHCQRqrX4H67qS4AfiWQ/fEgGvbWuSw3baXhiklJUVqUVGpjkEhl2T+uulnnmpWNUIPG7/BozkeUia4MLgdrPqkUzFWXz9yzYeR3fFu8jq8I72hwSuezz5UF9rIHI6zU8G304G70mEJxwGI2ykgxunwm1WZHixnhvvsyjlX7LQizqdA0yPvGLkdCwJSAokFrSTkFUNz0mUKfea9W8b3V8PKTyr1yl9kZLsG11UHQiDM/w3UBp+oUis3ekdupmWBeSJDUdm8QmPuXK+hFA/nD295gPWD/eJMzMkmM7xth+zUWLJi4eXA9ukkICusOzgJrbZ3IjUIrcysavmKFjoPjd3JTL9ay9vB3s2XrYsPAUme8eGW8FKCNI8amr8y/kIEMJ/KQqw4PdvpbKgJftZQHZzjchhPXH/xmq9FI4TxyJRSTYs7hvpQiEL3BsWoEgqcozhcKgxFJMbZdrRBFiySNGs9TnZ6uouFdINtrW5gedhvaxHOxJSChTgQsu7baO6T+MjUyRGQ5NnBF5gMPsOHb23UZ9LQvEqvGotm3eDO0C7y18PAw5Q1kDb/nJHTY9PUcx7igpgg22IM60vh02l6ch7Tma2lnCh1QzkXUhEOciu7LV2UiDhd+FSAMF8dcyYr7FOA1hrs4dfL2O3vfMn6Z7hYDX7bNlnSJ5bqGMpoXajikSE5dLft1UpIwyR5JgTG2CsRpFlc7OVDu2WbJuTCzEBD8+mZue6T6ir91b760EJD0f8yc9S67eIM0INqZWWFhqKQMUg4wYujx3xTsArOqsYgkzztK8B5oE3Y4w+xawYWsG8P7Mk8LtOTalxhf3b7v3YwF94Wtk6QPuHJQGL9oExXQ14tYwf5cs42F5ptjanNGyfJM8kXqFFitEhz6ZtF+Up98FeIrjszI9fds7bKv7220j8j5FGevOn81QeoeYJoI4flrTbbDASiF3pLXN8oajeF6xDtO2TPIhkZQZ6uHtaXE+e/dx1iQ7rnsMG98ti1Jxawv+04zE6Az2qVXaVPkjxNjAdqDn82enOQsbl09fCRqTMiIMkzSVdVEKDQad7/+g/7BAP+qUpD8IaMHFd+fXXgg/oq30zNF3hm2pUv+KlZmDnExS8vlzJjbOi8eURHxjeZGb1ZMNzNQAQ1JD6Mj7Gmds/uQau2Iaz90G4Z7JxBvfj5Hnnnv+yHp57ALhrah11+Sm/2t+0HmbPdW9KGE3C9J6RjDcplZvRBTV8wS23ShlplURsjIMmznS8OlzRZkVqsmMPzP2wa283BtN9gLvTsrBxXBnSZ79TVhhfNGL1mjWgKWcwRygMuSkf8NMWxqL+tOdZaWO+J8X0PQ+cGeZuSJ5qFOCU/OK0H61+Qwb4+VMse/GQvOy0/k5PnhYPBRdmfs/JQmGkY3tvYhE26BcUw9IH4fo0Vmga7qK+ZzJhSCJiKpZRqcuo0lvST5BhUDIPO/754bcayjRNnarr2WUa2+y704qOG6tCYG0euPnfGyGc4cVJ+IlvKT+Xa1NOyMk6UsFhnGqrqgfHCpnLnEY/9KzBkp/T+V9BWYBXkJ/67n1XVBUComWze0F6svCHIshFC09YJzVMMhYbtMnVyBCR5tsMFJhIklyDy0/SWFLmWb5wwpdxR81lmtutePFpf1VYFJkPSPG/aqFXPwcxuROZvM2k29udivqUaukdx8AGGTm1qPGyTjSyQ4XaQ4uSB+WxY325cbfjsp98wluZkrx+oYR98DdUhWj36ZBCrA44CeAnY+KaFIY+3LovDK4ftNrnL2qcZNr4P26fD7Vkr6TVJT8md2yE5Lt84+cqcgtC7mfmelYbCfDAGXTNj9OpzZoxavYraEUmaIIi2SZrNdAxSp5OpnDzbpf5mThv1bg/lDOlszMm9WHaag338zWH26O5KNtSDNkMP+tb2fbyu0wZ0Z1f3zWU7MLxPxxvGlomuISdPwyRPWzZSNipRCEjyTASSeGSICEGeOhl3VmyZfCaG6M96srTnICoNqD7sm5GeHT4d0uZTOB2XNPnwnELItRNpim6h+jRaKteOiRxOufUcPpwXkF4T6WOSNOsDWClEZd0gfT6x4xDbV9nAskCsY2A8ouG8y57dwZVFB2joYBh82F5EYa+43My/5Z9OhoA0GLXxghJpko6yGD6XK14t6q6HQnMU1ZhBoSEaqgP3ejzKPb+1XI54Xji1U/42nrbJ4qZFf7hSVlZmbEYO+igGw2RHe6YMDNl3Qac56/T+7KQeOayyNsDmvr+TsbDOPqyFwwFI8tCOcvbBzv3sijMGsgtOgiU+63PWgOMaytquY4LUFcwJQ+LR+/HNRyQRQVromEypj4AkzzivodeKdCTmny/fNPE6PRi8F76aBfDTXIV1aW+ZPnrdNqqeSPOIj2acJ2y9mEKSb2O2khL+KCtKOEB6QrsRKDXOZUlql0OipNHu4TofG9+/gHkGdkd7oduEFFcL388D+Bh43QyE3+etQ3qwBR9+zU7KdLNDJL43dtgGG+RbSzpZg1VRa7zev0Ir8VXgqOtig2bKJiQGAUmeceDIJUgacmP6Dg3RsQrtEvhqjq6p8H8BY9AUDM8bdZplRYWwtptuSnGcqtkiGBpa5AjPGGyT4enuu+8uCAaDxfjt9zL2OBVWDc0fQrg02sSQEpxklqMdHZlIb7kdQ/DLemWzU/t350Pek3pks3sn0NTwoxPXOuC1gJE8G3FKX7bgP7sZ+cqTzsleLwWsbWLo7DBTOHky9evnMPlhyJ133rlo9+7dDz/0EJ9aSy85ZThcyYqKiuCOhmsiU0oiIMkzhsuGm16lSSNEhuYc9MpbITHdBsdDo7bCf3uumvuHn498zEdVRhJsDKeIKiu1Aw8dlzLF9oIFC3qCOF/Lz88fUl5e/sKIsu7/uxm1aRrc8CmaEg2Cwaq0YYfE9ZaQKCcN68MKstysviHEvjhUxf04qX2iofSGCIMpc9LdbABcls46oTv7Ts9s9q+KOtYDEZcCIqMdOgWMdZBnjsLqTfZkpbhOt+m6ftOQIUOeRhNpxpiBa2avVtsDu5Rrha1GPXZGz7tpBGJjmqHflr8yaSTmnD+Qle8ZDmnznxhSzpxx6epPqf0WaRKxtesDsnjx4qyGhoYMPIjcHE2/fT7fDTjvbnDkM9hfT+3JXbT+dCyRu4Wpag7T+dzGDr/mJDGSOforxDz59OcXsqF989nLH+9i3//7WzjgRky6COiotZjzPnZwd/bUDy9gGbDCP7qljP3q5TI2JNvNDiBvh3cITbSSwTQnVN7hovJZY4ks2cqVK5179+4tENcJ3+RgMFfTtOqMjIyVM2fOLMdv6gKRqooPdT4CAPySyZYISMmzlctCarV5CA2Of6ElL34/Q3Nm3O1K16b5avX66vLAr+F69AhVQaRJATtEoI9Wqo36MD1QNMQrLi4WhEzDvutBnAsg1fx36dKlF+IBbJg1a1YNKl0oKrYeRD0Y1vc5NeWAojpyDB2rZnaw6Z1YIQfGnh0wFP1uWC82sEcuuJ2x5z4zfTuHuDRWG0GeyMqc2Lce7kvbD1Sx0wcUsO8M6slY5nZWCcORCxkQaK/jk1CJoDOID/IlNWjKlJXOqVOnkti/T6hWsrOzlerq6u9lZWWNqKqqKga5fsfKw3DN6BrTN5Eo36bfMtkTARICZGoGASJE8jwh96PlmyeNdDoyPoCj+zRfffApl9MzdMboVZw4KZ4md4hP8A1PDxw9RCBOss5zioCEmYntn+NzGOT5ByJOq/mkQ9NKSkrIA4jK8fx1t47fD63oYcy5pt0dTjN0w/GbDi255JQ+zONU2bb9lezPXx1mmSDJGrytaGK/+DQgn5uUnpA+3/jvHhxh7JRe+eyGE7uxg/D5TOMKUb67Y/+QRkRFz7CGO6L/HaLGPNRnL103UkrjUnHsFet6XVpZWUmOTH+CVEp5SKOiQGcN5S8nT51+07ZM9kVAXqCmrg35bZZiPjrNEMJwPc/IvdOV7pyD2Jq1ENx+NX3kqv+jYnyIDtcj0iY2VU0i9oEET1NV9aza2tpn7rnnHpIuGfblQhqtsUi1+dMUgUhLi8O5i9atRqDeCQjUSy40HTracOKOq4SPURp8OD/72QjWOy+dPfXm5+ya5z9ip2AYvj9C6hQdywBBfgPyJOPS3390AeuGof0/P/qKXfF//2KDczzsoD2G7jpz4E0QDr2vsLTRFbMvreLGueMNQvTMRd4v/Pddd901DLrR10Ca7+L4HFzjDwWBzkOQF/ymMpHlBDzyu4MQ4EJAB53blqf1UiQjkCER54qNE0/P0XPfzMzzzPHVBddrmuM0Ik4aygtpM1HESQ8KHhAHPjRk49cFD82tbrf7Pzj2QHp6OpdKSLrE8UoiTpGvWSApMAUSyn/Kl4lgRodf72wQYRDTMZeceQInTl9AZ/+gIbtbY1XNUEMdAB8MXeeLGLp/dbCSd/e7J/Vm5/bL5RZ7IlcbJEPh0r2xgxMnNajpZuFyGJhUUULXUcU2b3pOTg5AYP9wOByXQ0qdTjuJNEliRT4+hKdyPLP8YwsEOlQKsQUCEY3gRiHoNmkX/DZ/g9v6Qbzr9ZrKwG/gfrSS9lPEIxFWjn4nItFDQQ8J6uLnJoKkerE/z+/3P+d0Ou+49dZbuR6ttLQ0bOVn4qFqtg3W3GqY5l+DYnEOnuYOf/jqiSzcDvYJ4nX+6fn32QE4w6/bX83ynVj90yKSpvrDZyKBYBdt+ISd/8kuvlyHB1GWqEsI2d5UkeTv4/AaH/MTUzR5r8Kv57ENsa41H67jGtJhZdq0adX4vgES6DJcX9Ja8OtLhkB4UeTfdtttX6EcH95b5SmLTB2IgCRPAl8M00eWhiiiO36vyMxzX1tz2Pe+U9Guu2HkM1spG5c2Ezw7SBAnHqJcnOKnGKJvueOOO0jaxDOi/J7Oe2yK+uFpjCSvbDVCgSqmaDkgUWKaDiPRWpy9D0a39yO4Mft0P6xBKuuO9Yq4M9WxHY34TWzSE65JT++rZk9j1hHvAYIjk7uSD3V2sEiNO0jRDF99QHM43uTN3nowFkbn1wT3gHL77bd/QeWxTaOQEDwofuNyue6BFPow7olbcO0xN6DRx5eyxnIeyi9TghDo4HsuQb1oQzW4QTkGNEy/b+Pk78Kp+b3MHPe1mI9+vzrAf/4No57ZShIphCKllDvGt+FkVlG6+em89CEihBP12fgugyV2OY6dRdkwLKd2cWODkESt4tF/zZ3LH6yKvD174b79juKkVYLQww5M9LauQav6Y5G3ofnpbDBmCrVGnNRcAoOMR31dDnZKnocNga6zG7zmyX/ABjcxXJQcxOeHQiH2PrWXlRTFijMfnltqGXXu3LlcMoXUuQqjj6fh2vRr3CPnUtUgUox++EiFuzfRPpmSj0CHSSDJ7+rxZ7R8MvlNet+Gib9WHcpDelCvx1SWXxxlFEoQaVILLNLkD5bYxsMwGqRJrkfLIXX+w3ow6Npw8ju+5THsIes79KN5C9fezTzpcyAddbjRSLRedC7WmzDecuK8if82dMXlUY1gYG3l7HETElU/vWSte4Hum8Godw++uf8uvjPxqcM+giMx90qiGt5F6on1vu00sAind9JhfrMptDw73319dbnvPbdT+8H/XPjMlyRpzpsHJ6UEux8RgPPnz+8Ly2ovEKUppRyPauIeBi+MRF5Fz7vn2QuxFMcrmETu6uih+/HdTfk9YcWTrimBhl+Vzxr3KNww+AsrEb3C/Xecz+eKFSvcmEX2PIiVJNDrcR9tjSTaRJxX1tE6AjYY8bTeyATnMMPHYbbQfa/8qOeejaGXcwrc10O/+XeEjbuYiJOIlXTzCSTOxpcUpMwfh0Kh7bjZ/4YPx59ufPSR+2nSNz5CuGp7172mb+clA696E3q5T5iDBs4tWGbafsauVgNCPDmg72w4GNbVTbzzZaaXQyKAsO5Bfm9Y9wmDcckP3fimtLS0i3GOp2g/SagW0ULFJK3yicC+tTq6FHmSNIkbjEd4hzX9Aqb4/u1M00ZWlQdmTB+95tqp56yvp6G8mIbZGnitHaebmW5kOiflJYkBv3+DzX24+W/ADc+doenGxz6DLOn0jU8ikwFLl1ZarGBpD+UfiuZE3SahJvIkXbguA/6dmJNvvFY1Z8yXDNecJP0E48HvDes+4fcSpM359fX1Z2LfdLEf95mOT4h+SwJN8BVoorouY23HTQXjDPnLeY3lmyZfh+Dqj4X84fqgz7h8xqhVLxI25OOZwAhIfMiP81LVnBBJYsDvyfhdC6sq111ZNz7lab9UhKpL8VH1Fw1f7TxY3dPk0D1hcPMlh/Fe5hMn2NbhnNwSVvvxFfF7ie5nfD6kw0SUdB/h90X4Hgnf4PvwXSX2H1+F3JMIBNr7QieijW2ug0iRplhSRcs3TrrZk+VYjKV+P1YUZ9H0UaXbSNo0420mRuqzbmwd0+1ODgQC90LKLIWk8L9kSbWkSxa53eYORlMBDeXwgPHZRq60CYa/nqRc7k8aTXGZpykEMJHd4VZZKFiG1T/Pr/ROrASTcZybyp3ofZFeGHRfwUf015gz/xCmfu7AuS7BffildS8S4XLSTXQbunJ9nX7YTgYhIk66iZZtnPCnzHzX4oaawPMu1XkeESfpN8lNiXScbb0RrBuVK/jhflQIN5OP4KM3BsYhLmUWFhaK4ZQiSLSt54y6fHGpea0VdYURwHR4a6WyqMvLjE0iAAmPtCAPceIkQ5Gpgmkyb6J30j1kfbhggNHMwwg28hO0iSI1kd+wSMKlqUsIS6LT7f3dqcEUFvWlJUVpSkHwybweaRMqD/j+euOoVb/EDWaI4wkAmes1QZ6Nui5sD0S9MyB1Pgmp852OH0KREcHUdcJt6QXm8lxuBHxS+oz/4sNQRHPZg1/CyfMc6Dsrkil1NtFsepaPFQCa2tdEUbkrHgQI3E6ZBDH+cVNRr6AefCYz130eYm/eNeOSNXdQhyN9PNsIQOMNioDEvbt163ZIhBgT9XY8cVotsVxocu9eOw6+rGsR3xN8igcOCjvRVvkdNQLcPUn1++Ycnj12USLdk6JuwfEZG1/iQi1EEzBw/z2Oz314oT+CIo336/HF5Z5YEOiUw3ZBnMtemTwoYARfd6c7zqutxPIYFnHS8JqG6rEA1UxefiNSfbhJ74AL0n8PHDhwKeWlwA90A9MxknKbKZ/c3WZoO6XylvHrQJyItJRGskqjtJzcxqTw2bD2HHMgfp6/fluWO/Rn3hMsumeDHjW615GKiNqDkQ8Fy/bl5uY+DDe532Kb8jjovsS2TG1AoNNJHII4//Ta5EF+f3iz063110NG0bSRq54mVyXCKgH6TeF+RPrNEJzeixAVvAT6pldQ/bXY1xj8tg3Xpn2KWtJn3qLnTkUckncMRU0DkRKByocpesRhKEIY5lDwmoo5455iICN8mgwCEn2Vic+J+5Dr35csWZIBtyZyaXoVKiQKe4fNI+5Mtnm5Jx6Cdq2xU5EnGYdoWd8/bpk4DGuLvYygtH1BmGOnj1z9bKIs6uLGi7wqIM8TwuHw93CMlr8INZUnMn+Hb1sEWrBo3ULdnT7bDnE+OxyT6BsQgsTuYAHf+orZ48aZxY7ok6OvJjk5BYFGnE2MlnJxrNLaz/dF5JGbUSDQachTSJzWapYb0PdcpiqXTR+x+uX20G+CMPtjmL4fN2AgEmfbEyc1loZsMG7lLXo5x2D1b8DwUQh3G2k8iryQTW/rTEU0EiNcpWjaBRU3jfmEJiBQwOmms9tjL92TGLJrCKCNOLU8DuxQSJuv4PM4LPS3WZKnJNAYL1encJJvJM6XJ58JCXAL9HhpWAJ29I2j1m1KNHGSLvPTTz+9CzfcHASuvRh4vwoCdWzdupVmgaTGUrLkFYB+VBRfWtVt8drfYWniFzHn3QnHeTl8b+4BMg1rTMFsIhZkczhxWhJ8c0Xsst8iRz4istq0G98fIgDzLdDVk070ftzXKnT0qXH/2gTYlNdzCeJ8YOPkQl0Lb8BL1qmC1G68ZN0mOpYowxBuLI7VZ599VoSbbg5Ieh3I8z90HSl8mOW3aQejQXS3FhmP8PAfmjV+E0SO27DKmml5j65018tlsLCSloG1jv1PVcwe8yAHoDjmsHMdihuRKEmheNnXQvc5Fs7007DvPWqUIE4cS3lOSBbIKT1sF8S5fMvkIXoo/Dokzm6q03HhjRc//aY41lYg6WbChyQyPqyx9JtnDRs2bB0NgVJimN4cCHiQhFM3Zh49DV3eZISso/CaNAFepkYEEBHe6XQomPRgeMIXV87ATCIrWlVjlhTaaOKe5fc2jaronk6hrnRoU1OWPAU5kh9nQA+9geWyB+kh5fLpo555URxrB2T5TSbqbeImFIdS59vS2fVcsqpHIKxuZs60YXCet03MTxsAGaaoSdBzVjqYcSkk9fds4tPZVmgoUpMKlyaSREPkDwqni79AEr0DI6m1neLebitCrZRPSRGd9JgU+eiBVyYWBI3gCy6PNsgIskmJJE4xfMH3BVC2f4Wb6xpgaUyZMsWJfaQrxn1mE//NVi5yi4fJ2AGJY/9Nkw4gOOREI+D/SnFhkSFrPaUWy3b+g2EYiDSGYQ2u9DWcOOnadw7pTETx4n6+8Ael5ZJ7IajIGtzvI+neFs9A57/M8fUw5chTrFq54vkrsoOK8VxapvOMQEP4JzeOXrVakGp8UBwpRTcNPhTYYyj2voZPAd7KZZSjT58+tGolSWapo9+khreUjug/t+GVMF4P+A8izFrXJlByhEewYRjRyNH8x3BLeoFLnDb052zp0rZ2jO5zut9pgTnk/TbWTPoLlvxA8APGYJ2PjDHbWlVd7jg9ICmT6CJ7i71cJ6O7PY8h+vt3ayoC18+4ZPUTPJwcOzK3PN5OYShDLh28OCIi1eMNvAyfRzGU2crPb+o/463evuWIQLHiY+Xskf/pt2R9cV0ouN5wuDJZCB6zjKXUfZIAkEGc8BJWVETxC00pv3nc/3EdZ3HC43QmoKltr0IQKL53obZfUo3WsL1R/ymH8cfjnDo6T8wOIlGPZgchOtLinALPzdXlgflYa+h2r5dhwSyGaRNtlgYjdZqR26xTE2fkfWG53/S65/mLfXrwKcXp6d3FdKCmjlMPG/Do/MXhm8c9hotPfrF0+3We0UbkNbe26R4Xu7FNEukICA7DIDhw7wI6TvtFnq7+nTLkScN1Wr3yvo0Tp2FZ4OW15b6/U/R3uoA07TIBxMnvBeh7fo63LM1Jf4TetrSKJc0T7lI3jUWgBXevH6qrxipEYBpm+BtC8DfAMNac4trpHhzSXxsG5qxDXREOVqGrGKqPeY51EeKMvJ7C6o57fnZBQcHCw4cPL8X2/6M8UgI9glTjm+bILvttkfWcE+emiWM86Y7ltRX+t1V//19RS4lU20qcuDE4DjAKTYMP51+g47pCoECO7zjetd621hD+8C1jP9PcrtGQPF/HAmcwNoM4SRfY+RKuL1bApD6GQl9qivp9Tpz0EukCEuexlxMCA7/fMfFjCYjzIayVNBPPwJ2UD1IK+JOmo8pkexDEDKH7t4w/NRRS3sXTe1BVlXMR6GO3ONaWy2i9SakKki5fwHd2Zmbm+N///veHuvxb1pJAey55McMf9i3GNM7rOdahAAgUxpTOkbhhCEsHM+b3/TPfpU/54v9N+Jr0v8w70nbBPpIFeeS9j+diGSTQ6SDS+7E9DW04SqWVrDbZ7Ty2ljy5EQhD9UfeLMqHs8gqXDPMK1av4sRJQUDaGFYON4IIF8d1WYj6/hNcoNFEnBHH7HbNktcekkDhB7r/psvqKmePv0ENhX+ohEP7FFcaESdJJ6kshVL7Q3DLQl+MBt3fcDMiJF3BiZNLnF2XOOkGIwmTngHaxvcMEOdj2EfeJ5QMMqyam133r20lz0g95rINE0szc1xXN9QEeWi5RDrB48Zw4EMSRuPbFL+lYjzymbAeIgb1Rc6SdScqYbZccbjG4hliRpCkUI6drV/EEd2hFyVfLhh9YFjL6QNVMW4onz3hTZ4nBQJ9RPSl3Tcjn4VIadQ6seCPTm1Iaw5k297wpaXmXPLlGyfOzOnmubquyn8vxeTkF3Pk5jZJPFQHAQLjEOk2v8DvS/BNb1q+5jW+uc6H8sgEBAgP/tnkqLpp3I5KCsUWCl0LN6ZtmNKpMQ12aVMKtTNu9ICTC5KieDI0Fg7tZ/662W6H50JOnCRtki7P5hGSkn0/0rNApInzQvA0J4VgnwOrJvTEPsK0SxInXQdbkidJljQkX7550kgsTnhv1SHfxspRZ9xMDcaTTH/ivmC48FyqxEqDw1DP8/i48fkcH0oyqoyJQ9N/SQdIc7qRYFD5u6awb0NPeBsL6wcVdzpI1KFaBiXSI8Z9jZo+ebx74eUuSBNErxh6A/PV/VkNG9+umDNhMakkeFg5UlHYps3x9rV9ylmkaZAVns6AgDjnIRwj+T3/gn7TM0XfXS3ZrtOk56Spl/e//sM+elh/CmuUlXvSHD+lFTD5DCIv17XFe50UmjVBhREViaTXx2FZvxQXfxfpcPBtZ8kp3j4ntpyXHMUhieBBwnTFGugJFxia41w90HA308O7FCcIytQj0nkJY8I0eURq+vqahEnTg5xulRM706tYsOHPGK9fWDFn/G/Lb4NRSEqbdI2iTnhG+POBZ2YrpNG9sBE8SnPi6bnpigQqdBZRA9ieGSP1nPdtmPi8J8NxBdwLr7hx5Op/tlXP2YS+prErdOElcTbCEf0GSRz0MiKpDanHgrU9A6pyDaYz/BTG+DMUpxvxNBCkKQyVsmEIvTK9sBN934GccfcwEDtNqdQc+OsAbevQyfq3Q3J62qEqfzl489j/8s5xSWkuRKbOOWOI97Gd/ohnBd8Dget0PFcPYHu75Rua3BdlO/Ux2moTfRNHe94m84llNJZtmDQ9p8C1rKbC571x5Jp5XBqF5NlkoSh2CuLERfZg3m5/CJ2k52x8W9J2FNXILM0hQGS0FSRq6QsHejd5Ktw152Fp+DFgz7GgyhMVTybpGZkRQuB9XBD+MSVSkkrpPhT3IvmSRvwkXqTf1l+SLOmfmUhNgNx4jClIsWkAQs7wbmZoGzSDrQmw4Gs1t0w6zLNL0rRga9sXnpdmhQ3xrLXtDKlRWtywHd5a4bO54tWi4Tr0KbgI7/Qd5Tyf1iSKlEjjaCj1kT9suOhP4DG7CgRaiEAI39BQ3QpiHEe1sshxCBA5RUiidLzPyvXpDRXKSBDaxaC503AtvwPJMI+8zhQHpEPiPj2Mw1w6xZXCe4yESOua8QyYBUH5wZH4gtqNpErKQFItpEslHPaBT9/D8Q/Arf8KZWa8UHPDJSZhUkYantPqlvIlSWgkJBFJ4tnBKrTFYZI6y8rKzsQiiJ/edNNNdV2FQM27MCFwxl+JSY6luAhF2jfB0EM0i10xtClEnLRPwXe8tYMgVSJIWNavxfKr11RUVPyJiJPqo9lD8dYryzWBgCAnbrUuVYmw9kwdW4+cz/HPyvec+ZVfwUqrDjdC+tmQQk/Bhe4HVizAJx95svBJBwm6+dCbpo6FaLSvB8GU9TDn1RrhcAWs/Idg+NkPJt0Gsv3QqRkfOeode/d4+bmQHymiDUKtYB6QfxOBAIQQ8k7hbzkQZ2/U+XJdXd3LIM4f0DH8bhRaEnE+O9ZhC/KcB+UTFFDG3u7B23LyPedXHvRNn3HJMx8JaTRe4OgNCNKkNyRZCLdi2YGZvXv3foDq6ypvx3ixa1M58+GxXnhkXAKRUio+J1jO2G5s0ecl2kWJz2AK1XdXFS0jrGi0/pRH18NuQw/jOVQCuqL6NCPcoGvOeqXef7jSi0juTSXy0SzCATJsHNWGpjLLfW1FQJAnvnfjOXsUU5v/H7xY1qHevwuhpa3nkOVbQIAIkg4ve2XiuX98YzJFTHpZZCeJVGzH+k3k2EKZlo61UEweahMCdE1oaE/uTkR0wtodc6WmtZ/XwZfDQJ0tX++YzyALRIeAeM5o6A7L+w34FFJJy62pUz9nHdo5c7hOQiBT7tsw4V+gytMURT9t+qh120QUpegu4dG5hFS5ZMmSDAwlTqRYnJBgaJErGsLTUJ0PN44uJX91GAJEfBghcKMTSY6Uyrqb9+bwg+a1KsW+wiKDzcW1M6VKnk3+sQUCdK2afKbEs2iLVnamRgipE2HmZjzywQ+MZRsm/5b6R25Jbehn4wsBQ4kn8alctmxZLtUn3pJtqFsWlQhIBJpGgNZC4s+tJYWeTsKLlbXxmWy6aGru7TAneRH04/4Xi040dGNhTYX/XzNGPfNngnHuiPinX+ICcjUAvq+GgehHIMzHZ8yYwXVkJH3KJBGQCLQLAiR5cgPszp07u+O5+7C+vn55u5zJJpV2FHkqmDHExfywK3S3J8MJ44DxewzbMQWsbfE5MYNIDB+2V1VVzQXOswhrS+oUx2wCv2yGRKDzIACBhT9fN9988z48b49kZ2f/Egak76OHPG5E5+mp2ZO2DI/jxgLO8Cq5Ia3YPH4sFnD7YX118I8zL1n7GsCH35g3brckIkgkKk9DiA/xTR9OnNgviZPAkEki0H4I4BHkz6BRW1s7Hc/c5/i9k04XIdS039k7e83Cgv7XTT/z3Ldxwmf4HLzvlXEUoYUReba1/wsXLsyjC4h65Mp/bQVTlpcIxIGA9fwdV7K5/cdlTJEdbSarWPspQs1VGVW/zchxn6Iq7Kbpl6zbT1MzQZ5xOa3TTCFqB8pf7ff7P8FQ4Tv4KSRN8R1rU2V+iYBEIA4EaJRHRBnxXDqwjHdBZxv9JZU8SbKkUHN/3FTUC9fkjrpK/7vTRq55jK5PMYtvtg9dJJpBhLqpzidxgcJOp5OHmIPVLy4ypvbIJBGQCMSPABElyJNXgOnQl2AZ74/wjF5EOwSpxl+7PUomlTy3Dt/Kzd0hIzgtI9uVi0g3dxIMJHWSsSgeSGgGEZXDDCJM41TW4fPjOXPmVNAFogsYT52yjERAItB2BDBtkz9/CMRDwkx3PI9PgkA9JOzgt3R9iRZick2ivPdvGXvisg0TgstembCpsWz8M4lofMA/jXWZG/LCHAOI/CkR6AgEhJQJ0rwas4/WLl68mOIXCO+XjmhS6p1TkCcc4v/84FtXGyteGX8h9UI4yieqR0SmiapL1iMRkAgkBIGjnkkQKYW0S+qoNyG96IhKBEE+sHFy4YotkwwEOi6hdgjLe1vaBOPQ7Xij/Q/VIS9IW5CUZSUC7YdAM4R5FKm239nbp+aksH9RkWkMChrhW6kbsLAvoO9ia5E32o4liaEASLMYa6zfqev6KbGUl3klAhKB5CIA8qTg41wHCjvFrXh2f4EWYKCYuiPFdidPGq5jVqSxfOOEs51u7UfhkPHItFGr/0P7S9u47jpI81LMItrpdru9yb0V5NkkAhKBWBEAUYoio2GBp/WPziSjbqqOGNudPNk8rwWY8j8koytO9X7aMbzUtLxbB2P6KiwsFFdhDgp+l6zr9Aajt1tMFcnMEgGJQNIQEJ4xeFanY/VNyD46Lf1Ns49SevjeLgAKI9GKDRNOgoUdus4Jq/mJTOt6wgBLZdG/XYCXlUoEbI4ABJ1BNBuQmpmqkme7QiwMRcs2TFzy4NtFIM9J3ElW7I/35BD3T1+0aFGOBD5eBGU5iUCHInCc4JSKAlC7DdvpbUKzibCgG6LaGlN89aEt00evepUuGfbHPLwWRiJY16+EvuTDYDB4KdUlRX5CQSaJQEohQLOP+JRqkj7BFQOsCS3Hkaqde9Vu5Dncmk2kh8M/Tc92ZWP+z1ICwpI6hc4yKmzorUSzEijIKvQki6Av8QPst6iwmMUQVUUyk0RAImALBITdAtM2b0SD3gKBYiFAllLGo3YhT/Lf5FLn21dkY2nYmXU1gQ9/d/EqWhiKFVtuS7QdbQJR8qxElEjkI/pjWgGTpFt8YpZioz2vzCcRkAi0DwJbt5oGY1VVX8fCcX3wPZfOJPa3z1lToFah01y+ecK1D7//A2P55ok/p2aL/XF0oTHEf0TZlBLxI9otNyUCEoGIue2wws+HHWMmgZKKus/EXcyIeeowFG1AvM6KlZt+1I1OQJJiHCc6iiRJV9K1AY4DQVlEIpACCBA/WKtupkBr26GJtOolVbti48TTzaWEJ/J10uOROgVJUixAvJ3ugbGIzySKk4TbobeySomARKCtCNBzfswzfZTA1Nb626t8PJJgi20RK8eGmXEN6SoVpv6NCpQVNTq2t1g+8qBwqoVl/UboRW6CsYiUytLCHgmS3JYIpDgC9JyDPHUM3S/ENqn4YjIod1T3E8rwZCiiqZhLXrw2w+Go3ga1xjczRq/5Lu8cDedjiNlJbyMkvnAUvstQRznWX/8e1SWO8XrlH4mARCClESBVnBXQfFF6evosWOAvhEH4dZJGiVTt2rmESp6lrIjX53TUXoklNvpqiraSOh5PsGMiThSlN1IIZPkzWON+THURoNYx+imTREAikOIICLcldOORhoYGBlfE66hLXcmHu1GKRczOFzAds4oc5AkEIjz6jjE11ifKxVmPKC6/JQISAZsjALvG5Ri+89Fql3neRUfvfWXiySBPY9nGCY/SdRLz2+O5ZqJO+hbb8dQjy0gEJAL2R4DUcaKV1vPe+Fvst9N3PBJhk+0fPlc4vRoTsBY7BtwObiiKI3oSB2zFihVunGg9FMjXA0jb6j2aBEPulAhIBGJGgNRxwhXReuZtbThKCHnyGUUKX9SJrDnj6msCe7SGun8TemVlsVnZ4efF21ReXn5dfn7+laiC/+7yMw9ivhVlAYlA6iEAw5FOJAryXIDh+xTqQaREaqceJYQ8RUT4+zaOO8Xh1M6Ho8Gz0658oZp8O+OVGgHggIqKio8B3OMEWIRS2U74ybZIBCQCCUQAz7uo7RK4Jv5pwYIFvYlM7UigDtHStnyTb2cpr0C70uFSWThgPE0/Td9O8wg/HMWf4uJiPkSHy8LCurq6P4B8qwk4JDl0jwI/mUUikMoI4PkngY6WJr4Lz/waWN7PwPZey+e7kVnt0MfEKGQtH85lGye+rjD9pGCwYfBNl71UJ/w+Y+gotecogCziPGpfDPXJrBIBiUBqIcA5gHSfp5566hmQPj+FAFWfWl2IsrViOuYfNk0einnsFC2eL7OBDselEkA5Bz5tmQsfZctlNomARCCFEEiMoJfADsdFcJHnF9Mx1XD4Mk+GkxmqsZaOC+t7ZN6WtiPI9jfIt4Hms2OfbkddR0v9kMckAhKBxCAQwQlUIY0+bUWgbSVPHreTeqaqbKyvNrjHcLveoN9FLKZo8XxuK7knQc9xB4rnYYpWFdWD3/Qlk0RAItCFEKBhOwlP1nz3NWQ4QvdtFSy5TeQpDGM0k0g3lPPh4vr6zPNKG+iNQXPco73WkC55VkzNcmH7K/y4E3WErDdP1PVEez6ZTyIgEbA3AiBP3kDoPLOys7PHIzjQz2lHp3FZFGHmYCiaRAu8Ld8w6VrqoNhP29EmMTwHYbqiLSPzSQQkAp0WATHkpFHpu/g8YbeeJsRVCV5YI0OBMNMM5ztxdhCjcx4IhAGkQJx1yGISAYlA50FA6DgNh8MxCS5L5L5EiUjVFqNRwe68VbH8EW5I3pIiV063EGYTGcG+h5zn0NpF4lgs9YE0Mz0ejzZ79mzSddoGoFj6IPNKBCQCiUWARqRCsLJqtg03xK3zLC01w8/l9Ayd5HKrhQjWuYmIk4bsMeo7BYGv9Pv9D1sAibdOYq+ErE0iIBFIKQSIOCFYRfKULaROAjHuYbuYN6TqyvmqStYhZQNVGMusIgIF4Oj4Ptnlcl0DCzutjEnJNm8Xsznyr0RAItARCFiSpz5//vy+GLrfhzYsBl+8R9yBT4fOOoxk9JiwKbQCfsAadpWvPlRneDS+jvpcBKGLqSIzczqI8xNsPkg/YWmLu11mdfKvREAi0BkQsKZlUnAQaPU8V6NPM1K6X6TTpA6UbC1yIW7nLnxebuyQdazxdysbwsq+dOnStFayysMSAYlA10NAqPUYiPRZfN63IGjc31GQxCXhiShKe/bqQzVV7Q9955vUAe6iFIN/J4o0KoNnzpzZ0FEgyPNKBCQCtkUA8pUZJFnTtKnY5u6QaG08I9yEdjIu8izsfpCzvqKGvuXOcDDV0N+jVpVZ+2NooUHrNK9cuRLRk3nq8LdJDG2XWSUCEoEkIGBZ2xUsCvcN9JxlpO/EaTucK+IiT7ZlBFfUwong3PrqgF8J6aSvZGI/3279D+98WVnZPfv27ZNW9tbxkjkkAl0ZgUaru2Uo6nDJM/aLEaHTXLphwr8RRYmWBRYpqreB9eZgmK/aE9sEymtWBVQ+qjrECeW3REAi0PkREJyxaNGiAdB7rsDvAdRrMaTvCARiljwF3T/4xsQe0HV+ixnKu9Rwq3PicFR9wXzVDIjk+5H5USpgWdljqiOqE8lMEgGJQEojIJYhhldO35ycnN+hMz+lDmHZjpg5LFFAxHzieczLJUOfzzjbnaY5MSfzVWrM1uHmAnDRNAxEK/yzdkIJ/G2U+TuVo4Xvoykv80gEJAJdCwGo97hQBdfIjysrK3ej92cTAmJ/R6ARu5P85s1EuLqqqOcocI5XQxpf6O3IUhzRdcMSt2nu1dfRlZC5JAISga6KAAlcxBlItdj+saqqfsIC2x02Uo1Z8mQjLGORrp/uqwtV+XX9G+pEjKtkEgiEBO94R+otqO0ySQQkAvZHwOILirL06h133PEOvkn46zDyjAmxo5zjN0z4eNmGCY1RlMSx1ipEhzlhI8jpufj8Fb/5khuSQFtDTh6XCEgECAFyb4xAosMMzDFJnkLf+c3hcAEaPwQu7v81OxNbMBAqA7K8DvrOn2EzFx+aPdBhIND5ZZIISATsjwAJWVhhM0zCFzhjMlpMkmeHcEdM5Dm81DQKwaO9d1qG041B98cxws2X26AyAOFkWNs/hOS5nX7jWxiR6KdMEgGJgETgOASEkAX++CUOPgXe6IHvRh/Q4wq0446YyFO0IxxkheGwDi8lhTvHm5GUxNEWvxvfElD4zsCHuxugRIe8OVpsqTwoEZAI2BEBzlkgz39lZGQ4oQc9lRopXJmS2eCYrO0i3JyhGKeGAgZzO9VPeWPnxdRkPiUTovd/qBSJ4cJwFFMtMrNEQCLQ5RCYO3duGNImQ3T55+rq6uZB9fc5gQAfcXuPXNFozvr3bZj40rINE/cvLSnikZCiNRZZV7pRyjxG8dvlbgTZYYmARCAuBBo5hEqTABZXLUkrZE3LJAKFlX0HPjz4MT9/xJTNVtrDOwm9xUTUM4by2r/jrfRIHpYISASSjgDxhhDmkn5y64RR6zy988yZRfkXf9QHDNgN/vHbqA5I0Co0lq36WlkdNZYsWZKBjj+AojdSeaEApm2ZJAISAYlAFAgIVZ8LvHIGPjGpH6OoP6osUZPncGv6pWHoPTWXmol12r/kZ7h4RFR1iPWWfT5fH0SE7oOyPKBIRyh6o0JGZpIISARsiQDUfZxzYHA+Gyz6Dj6jqaGWgJa0NkdFfNQaEasTS2P2cbkdTNHYV7G0Eh3mCt1wOLwXC73NRlkehs72it5YOinzSgQkAslEoD4rK8uFkewl1kmj5rNENDJmcReRlPqFQ3BT0pU91IDhB7u3OmSnfJZFnfQUtfi5mPYhCfHb/CX/SgQkAhKBVhAQAheChGyrrq5+Gtm5ChGj26i4qJXqoz4cPVNbc9rBd4P8DaGAZuj76Cwx+Hge1ShLxE5qZ49qgPwhEZAIpCQCEYJYPTrwA3DJI9SRZEdli9bET/k40cFNaRWCKp2blqOfNPWc9fXkphTLOu3o6ADURZFRyvHdWC+2ZZIISAQkArEgEMkfkdux1BF33ugkT0s+NP05jYHgy0NEnDhrVMQp3JFAmB4UeAkf0nky/KYOyyQRkAhIBOJBgDOTxS9JH8VGRZ6iVcveKvKAOE+A3pNb2qMlP8wmEuc52e12n4LO8iqFBT4e1GQZiYBEoOsiAO7hnHL33XcPRZCQ9+fPn38eoSH2JwMZQWpRnUsJhXMRgjMPg+2dvMDcqIqxwsJCwb/VcFV6G6X4Ou9Q/EZXgcwlEZAISASaQCAUCuVgjvuZMB5xi3syBbKoyHOe5SCvhYweiB6PYEoGNxY10Zcmd+FtwN2U8L1zz549F+H7FcpIoaWaLCB3SgQkAhKBFhDAHHcukLlcrs8wx32vGM1GCGotlE7MoajIk1kSpq6zHpoDRRR1L51ehKiLpSkPPfRQMJb8Mq9EQCIgETgWAdhNOHnOnj27CsR5HtSBKygPBDMxyj22SMJ/R+XnKUjS0FgPB8jTCOicPBPeGlmhREAiIBGIEQEa0VIRkCg41STVGKuIK3t0kqdVNZrVg5yLggo7HM/ZoNi9DXPZuRxLHY2nDllGIiARkAhEINDII8kkTjp/VOQppmbCv7N3wBdmRlivpsKl9Kf1xDs3ZcoUJ5S61yP7+NaLyBwSAYmARKBlBCBxEn8ZEMpOh1A2C789VolGQm25hrYdjYo8xSkwJbMHpmb6XXqYpliywqJGK7rI0v8K8qAAAEAASURBVOx3nz4UjIkFIHF+QJlkNKVmoZIHJAISgegQEPw1Oi0tbREChRRSMXjxiP3R1RJnruhOsmUEt5YzVcnCeWqCmW4fP9+8qM7KFbh4KwSQuwiK3buoFH4nTbEbVStlJomARCAlEYBAdpAajmF7Hn0nywWydfGWAh1b8TqXbZzwMiZpDgqF6k+76bKX6mKZmnmMMpfOK8mTrrRMEgGJQFwICE5ZsGBBb0Rrm4ylOf7vlltuOSj2x1VpDIVaJU9BkPS9fOOEd/Dt7FvuPKe4uBTR6aKbnon2HEuWx/6Oockyq0RAIiARaETgWC459ndjxkRvRDdsx1kf2/wzN6gyCzRYQ8RJDaFWRpF4ZzCNqgB6zqLFixfT0J+kziiLR3EGmUUiIBHoqgh0yLLDBHar5ClmFwWy6tyYWZSDMXwVFfRGufyGUN5ijfbLodQtwTdfKlREg6a6ZJIISAQkAvEiAPuJvnDhwsFLly7NRx1JE8xaJU/RIX89c6FZmfjNLe3DhxdFJTkK5S30EL2dTieDu5KckilAld8SAYlAWxDgHARBzBUIBF6qqam5hSoDmUbFTW05MZVtnTznmqcIs7ADLaKQ937ac8T30zze3N+ysjJhGHq1qqrqgYaGhu2UF6RqWvCbKyj3SwQkAhKBFhAAF/GjO3bscGO7Jz7DaAdJovhqdwKNanomNUj1a05dNZxMNSiOZ9RJdATf/0Ih+lBK6jQq85Tyr0RAItCZEBAzimbNmlUDe8pz6NvH1D+QaFL4pVXy3Fq6lTO4ogadqqqoGHQ3xHEBhPSZtI7F0UZZRCIgEUhRBBBlqVg0XZCq+N1e362SJ0XcpGmYgZDmdKdhgqauc/KMduE30XBInnQuiiRPOlMi5EZCFXnkt0RAIiARiBOByGF6UrildZ2n1RNVDTk1aD0VQ4lp2A5lrkZVYOrU7/H1AkiUjE6QrGVgEMJFJomARCAhCBBhik9CKmytkqjJU3EoGobtVB83GLVWsTgOgxEvBLI8BzMALsD+XDom57YLhOS3REAiEC8CQgiDUHYLOOXWeOuJp1zU5KmFdJqmCRHSiMlKPnz4cC5Co5PbEDL/Q9RQSQ0VkaBpWyaJgERAItAWBKAO/BnK/xwkKjgtchjflqqbLStO1GwGcSCsOrDCMP4psZGnWGqDop6grivQOe4nmiylrmi//JYISAQ6NQL7IaAdBL/EJNy1BZFWDUai8rBiSp5GOL5IzeROgLroQ2+EpCh0Rdvlt0RAItA5EYAQxjsG4vwd7CpC2kwKx0RNnqqimQ2LUfK0LllkZyRxds77WPZKItARCBCfKJA4SSVIKZJrzD3t9DfqYTvTw8iLdsLVM5a2WDoImrw/Ap/ZK1eudFrlTTKOpTKZVyIgEZAI2ASB6MlTxdqZIHXd0KMvY3aS54d4XQxr+8Kvv/66H+0GkUrytMlNIJshEUhhBLikOX/+fFrS/Az0AyP45LhBRk2EhsoQv9NAMHlFSI6x4s0NRSBQ7veZzMXpY22ozC8RkAikBAKNQ3R48jyMFt9OrU6WMTpq8tQUZ9jQKaw85rfHkOCSxKMooXOPI9rzL1B0FxUvLS2NafgfwyllVomARKALIEDCHCVInLTwWx+QppvvMP+0+8g2aoNRsEHXnZoGmRih6WJI1ltAueuuu7aiGH1EkoYjgYT8lghIBGJGQEiYIE8fHOQfxG9uNMJvFZ92F86iJk+HB8N2SJ6QiWOSPC1EjiXKRnE7ZsRkAYmAREAicAwCGOHeLHYlgzjpXFGTpzPIwroTHGjopuRJEUNiSOiQA8GQhyKS/HZs0+qbkkBjwE9mlQhIBOyFQKs6zzJrbXYjHArqYTK4q+nUhbLNB6PSKYAo+TlAnKdB7/kfiNZXUXmxPAdtyyQRkAhIBNqCAHgmJnViW84lyrZKniJj0OEJwAMgCFdPWsAt5gRjkdPj8dD5hsdcWBaQCEgEJALHICAEszvvvPN0HHoX3+dSFghm3KPnmOwJ/9k6ec4zz+lwsgDG7A1YBC6D9kQbz1MEAHG5XJ/5fL4tKFpG5QsLC4/Vg9JumSQCEgGJQLQIcP5CjOHTc3JyToP1vQcVBLdENSqO9iTN5Yta5xkOB4NwVEIsT8MctpeVRkV+wiI2e/ZsWnVzhGgI3hrtbg0T55LfEgGJQOdFABzTgPXRyjG3na+Php4mhVtaZWi4JqFtzFj53pT0+qqDWCPE2Dlj9JrR/FKQ2yeORXlZ6Fwib+R2lMVlNomAREAicDwCK1ascFdXV/e47bbbduOo4JjjMyZ4T9SSJ3ufBZXBrBYt48N2age1slX2PdLgyE5Fbh/JIbckAhIBiUBsCCjTpk2jAO1fx1as7blb1XkKcpw69SEM240qiKE5f900gjz6o05CsQtH1uvweSxi7qmoPuq6ZEaJgERAIhCBQKQgllQ+aZU8aVhOQ3ersRQFPreS5XLynDfPK/ZH9KXFzW9jbjsR6EmUC6Qaa/kWK5cHJQISga6HAPhkKrjk++g5uCo5QUEI5dbJE5nmbR7BTf9Y/K0CY/VcpqlYRxNpLv/b6h8sxSFI8n04ydPqmzFJrq2eQGaQCEgEuhQCgiQXLlyYh+0/o/M/IwBApIJr6Ge7pqjIU7TAUIxyRWUeLahzi7tY010cb+4bS3Fw61d9ff0qWMRGI99nlBdvi6RYxZprl9wvEZAIpCYCgiQx8aYbJuCQVfvTZPckKvIUPp1QLhx2ujUW0k13pRhmaJJeQlm8eHHVHXfc8Ra2JWkm+0rL80kEOhECELy4rjMjI2M3CHQ6pM8nqHvCrzwZXY3e2o7WqEzZr9JqHIq5fHCMDaRo8jzaiZQ4Y0ROZpcISASORYCT58yZMylOxnLrIATQ+NZYO7byaH5HJXmK+e2QjffrIQRE1lgfqrx794NKhDGpxfORjoJIE+L2aHxiEFpbrFYelAhIBLouAqTf5CRKgpnYThYcUZHn8NKtXAmrGOqhUEjHUhwqJ8+RIzeHyIE+msYKHQVI9H+Q/+ElS5Zwf1Ei1WjKyzwSAYmARCACAU6ciBM8DMQ5kAQzHEsql0RFnsXFpTwavGaEt+lQeKqGzueQPvDKxIL7XhnXM6JDzW6KZTdIsYs13HMwz50TsCDVZgvKAxIBiYBEIAIBS+Ayli5dmoaAQ2txyEuHkx2prVWdp9fwqj3f/E83X8DoAwY9HaYeiq40edmGCYOCinEWQoK+jHb/j5jGSZ1oKpWUlOggTgZr+99AnD63272P8uGN0bLkSpIpdz+Yizj0pS2/WQqLUNc80hrTpNKW622qkXKfREAi0A4I4Bmmp3EeJMPYnmFqy3HPMZbwIaEvjCmZJ2VmZg6pq6v7J2WkgCA4RptJSc2SpyDDHq+/m+MPOp8DaZ6pYyY72EtxuNQhmkMdQpb32urASmppcWkROmRKqE213FLkKrfffvs2HJ9v5SEyjAAHIHvhp0V+oWXdcWyLjjOSOI48XqtIC18CNy/yerFkHdussq0HDVZSRPVQwYhztVCPPCQRkAjEiwBNqzGJkp6/4Xj+ipWwNaBu/fmLfIYZ8cFmB6+jrMxgXghFTDHKaNtMlXB/fB6S6Crrd1K9eDijNIdSUUmRVooh+7KNE2d60rV7fXVQeJLR3YDaE6sQOxyqA0tzXHjjyNWvlyCvGN43Vx/th6R59PoipqJXhbQIgI+XFgeveN5dXRPM9enBbIfmcOqq5oTVPwMXKBNcDh2CXptuaHUNih/qWJffcDsqq9+qrmKlxVzVcFRbWjnXUXnlD4mARCBKBMAGJZAGifiaeu6wO29RSY7hyMgNB/Q0h6E4VLeeFgqyLE1RnRBz6h2aWhsOhgJBXQu6tVCtq0Gt3OMdiyhuxyWcq0Qli3NpcRPP+HHZ229Hi+QppE9EVHI2VB14x+HWzgz6wyAlRdEcihoO63vx53s3Xrp+Fw3vvQpX2rbaWq/3D92Y09/He8stHx9FmN5Nju5u/8CgGjgXs5m+BYLsjVdMX7y1BsCs389QFA+Z+i0psvE8ZPJHpHv6XQNu34lfu/H624vfX4Po/52lsfe+njV+T2MB2igq4bOmLKlUvMmOyiJ/SAQkAs0gwNVp1jD8GMLMXvrPfCWsn+EIh79jsPBASFq98cz2RU0n4pnOxTOsQn/HKyYC4g8fSasGZDM9HMQDvg+TcXYyQ/0Gcto+RVG3GUbwrVxf3rad3pHkmnQkoR1FGMZ3BJG2SJ7UwhIDEqVSGl6+cdI1mkt5AuRJ0qfuTnc4/A2hzX0POS8hiVMQ7ZFeHb81ZeVK50NTpwbvmnfbjZDI53cLfjX0twW/aRgQ3H9VtaF+FyheBAQHYKmPHMWTgQoAaCgETMHX9CGAuXRKQFv1m+ib/QDiDCt8KkSwGjQSuBi6ryEEIq1ChjKM3V/FuvPvKOnalvJpV1Y3trAERFrEh/ai1sZDckMiIBGIRMCSMiOlPgg9ue7aCzAYPRekeIFuGGdBoMlTXB43QxR1BsHGIJlLh4BDi0jSw4uH8qhnmMb1RMikXgOxKiqeX3qGsW34MaNbD9Ug/34UftNQ1NfDeviNPzn/9NVPb3qpjreOyhaT9Js8adQknUhsjtmOJMX7Nkzc6PRoIwO+kD8t0+luqAv9ccao1Te0OmQncuI6C1MynXnnvdMaXBnLXzPy3/1ESetW4PacCMAtgC2iBG3ypgBOgEbtNIGlnQRUZDqaUK2rwy8NldG4tEoXRHMyI+jHYD8EvauyGdLtKi07882D14+s5dWRNEpGJy/Xs0aeQW5LBLo2AqTyIluEIE0QZn5azTl4bMdD53gZwDlDSYMmDWPxRmEHTzSeQnoe8bzSMwtm5E8u/TzmGSZ0jzzH9MRTSSpLCc+wCmuzJRjhO+ivPzTQ8FeAVZ/4xJn5GPv9VV+ZWfGX+KYYwhD0o4372mGDd6W1egU5Lt84YTTAepkAUCDChUPGr2ZesuZRcfy4eshoMxcAWLrMrD+sH+oO6b+uNpQfYE2Pvm6Xm2VAosTacmGrIfRlAmwS5nFVxrHDAhAtN8FUFaebzP7mGzEU/Aj7H3OlGU8emD6e3mwm+BFkH8c5ZRGJQOdAgEiOrNsWafZc8mKGX/dPggT5SzzXFytOivEDlgvw0TQND4kdqO/ms5wIFIg/6PGlE3E+xW9V0wKQamnVNyXgqzFUdRVUBQ9X3DL+jcZTchJtP0mU97LxZC1soO3kaWTct3Hik54Mx498tcF6UOP5M0au+fA4I9AxInT+3avO1RVtKqqfoLjTcujtpGGqEsmYps8WjbeTlMy3m86vARm/nC70Cx0L+nZjEsDjuoM9WnXTuB28Ne0MfpJ6LE8jEYgPgYj7P9tbkq96PD8GeU1lmrOQVGMYxRGZmc8SxBE8U1HzSXwNOqoUnH+IUkHWiqYqEMQwvA9CqHsZM8gfPHTzuGd5btNITJZqUjcmNEXdWWF5X75p4rd03fgAfLM7GKz/1k2XvVQniJW3LALwbovXnhIylNno47WKO13jbyedlJcA2no9JbQ3cVVGWmokGPEVp4suQDl+PcDc4WWVMyZS/FIAj9eEHMpzKOSfLoBABOEU4Xl+6Yv0X2KcebPiSjuJ6y5DARID6bkxja4dD4nZHqjoSBI2oFtVwsEXIZotqLhlwmu8eSYvUZspb0JS1ORJZxPDc+g+/4Kf/aaPXk0BSM0UIW1295ZkBj2emWjnTMWVnmMEoPA1SNC0E2mKhjd+m8BiOEBvMeZr+Ayj/AUVs8f/neeAjod5R1IfEgZ+45nlhkTALgh4vbjPvdze0HPhulGYCDNPd3ouYDoej1CA7n+yI2Co1oTOsuP7IEhdxQhXAe/oGOA/Am+ou8pnXbmbt92LdidIEIppuCwChKgO9X4MtHkIKJI6uYKWhsOwdOUsXj866Pa8AUvbPLKaG/56mNqQy3xLxUTWSb4WhIUGq75u+OpDhsMxFNLo3/IWrlufu/CFgSBO6geJ/zFhluQ+yNNJBOJEAKRCBlMQZ/c/bsrMW7T2T37FeEV3uC9gICEW5sRJkiaG57YkTuo38QusSpBw/A0gerCUO22KrofeyV285uc4ZhqDSQpNQIqLzDhh4uT0/mGmRBYiUsn1nAXzkHEHH/4GrbeUKW0moKnJroKG83jJutNVPVC/T2XajIrZY/7BWxGhmkh2q+T5JAIJR4AEAksn2GPRs+cFjPCDiif9NAgRdCqSNhNCNglvd+sVktAWhssTvNIh8wSDfwu7XdOrZ15eblrk22ZMios8RZtHgDg3QyLrfs/6IUHd+LPi9owy/LC6mUP0VAVcdI/eDmTl0wE++UiQHmVlhss3Y/fM4obGl8aR3HJLIpB6CEQIAnmL1s/G7T4P3iguGIOINGmU1SaOsAcgXBDCFJsMhfnqPoPf028PzRm7mY8i2xAHIz5gILYLr/68e569EErkp6Bj6AtRmXQlRJrx1WsPpI9vhcLfvtCjpCuGr2GzSzd+eOBWuDVF3HjHF5J7JAI2R8C6f8ko9PKXnpVQtf2SG3U7i/ATCT+NkjHdCYE5HEYoEMBrYUrlrPGP8ywRkndkkda24yA5UrjOw8er59/zbBFUhI9hFkE6lMlEnJgS0GkTV0bjBiOvgc8VxXF1xeyrPpYE2mmvd+fumGUY6vOH9d0aQvrfmSvtMgg/ZDSl1Jn1+mH4eGuY8gkDmD6n4paxi3iP4yDQ2MiTFMW8hGLkLlr3O5x0BQ1nYWRJZb0Ixy6GPyEQqAN+oYcQlmQSd4WQEmgM8MmsHY6AZafotXD1QJ+irWEuz+mMDCzJ99XsIChoGI8pS+QbGvTfXzl73DTekBhdEmN5wyCuJlEniPPu9dMUh3MFxGAQJ4+0lPr6zegvowOSZwjifzcEKlmfv2jNeXz2BRGoTBIBuyNA9ynsFAVL1/f1K9rzzOkm4jTVbcl1cu9ApEjiQzy4gE+HuvF38Kj5I28MuTCRBBplil7ytKSr/IVrf6FrjkfNQB3k/EUN6ZIpxKA/gQvHYcOhXVb1+zHvA3gHPtxHrksiIjttbwSsZ7jHgrU9Axr7J3N6zmAQBNDozqxua+makPhngEBVFvQtrpg1DhN6aA+NsGEsbiVFJy0J4iQdJ2N/g8QJ9WuXJk6CVcULBEN4dyYLhq7KGnHN+vq5Uw9xHWhpaavAt3Jd5GGJQGIRoCHp9d/SaW46XJHWgTC+08WJk/A1hUeakYSJAJ5RPwj7XnnqVRCnyjZvbhX/1qVGkqYQFCDvnrXnI1rK46gYxElh6bqsxBkJKobw/hBzufsGnUZpP8Qx5EN4ulFlkgjYBQGSpKxZNQHd91cQ5wWWZ0xXlTgjrwy3w5OXAYIU35WzeN0vMXrUIQS1+gy3nIErUL2hXiue747JlbCqu9MwTcv03I88fdfedtAbHEak0+sCgQc5FHSj0g0rk0TADghQnEuk3IXr5zJnWpHhr0No25R1fG8PRIEPdKCgNhiB78/+w7pvR2PHaJ48I95WmK1IMw4GwzJlKpbbo/mpXSckUFgrXZ5iKJ9n8a5YN2xqd0u2PuURIJUbpk3nLXr2Kth6vZA40SU+N1C+3I+6uBhJw2sIEwTS1KDxaM7CZ/PMUWTzBqTmyZNb1hnLuXvtzfDjnIw56rTGUHQ60qMa1UV+QA9Mor+hGHd1X7T2Ih7Rmm5cmSQCHYUAjRyhcstf/Hw/+L3/iUdmN6dbSuJs+ppoJCBCrXEqmHQ5z2JOW20Sr6bJkx56DD0LIL7iHbWAlsIwjURyKNo05nwvBUygN5czyNQHKLgCf3PJ4XsLkMlD7YoAIk3w+o3gMkxNHIA4unLk2BrgEBAhKOrM5bo2d/Ha63h2r7dJIagJ8gRBWlGjESl+AVjYwQPkde5ZB61BGu1xzD7yh2GBPzVYXf17XkgO36PFTuZLJAIkAMHdJn/husm66rja8NWRj0yTJJDI06Z8XVzY4TEtSGD0Zt29qoC7Hzbh/3k8edISokgFi9b/FH6Ml0JHAtDlcD2Gm0IFgSK7MjNv0ZpT+fBdrNQZQyUyq0QgbgSIAMhDZtHLOZjQDQUcRp1YnY2PHuOutEsVxPA9ACEobaDD4TB9P7Ge0LHpaPK0QO+24JneYaZ7MQxFfvyRQ89jcWvpN1wfDOhNPFmAbT7PyFf0kyqPlkCTxxKIgGWvYKzhZuZOL8RwnRZik1JnbBBDCIINQ2e/IzdN7up1jBB0NHnSQk9IYdV5PVj3RA669OeMDXLKjeERrO+4XR3j8hc9SysLIgy/iS3fln8kAu2FAHcvVPSey54dBKnnehhA6ExNGjzaqwmdpF5TCHKlueGmeTPvk7mscSOWR8iTxvQQ9Xvc90pPgH6dgUXaIHU2ZuwkgCSnGySpGwxr3jkxVtKv5yeldeFlkgi0NwLDS/kzG/Drv0QIxRwraM+R57y9z9+Z6ichyI+A0Aobm79k/bm8a0VHnOePgEprMiNhPeSfwaezH9YDls7wbbsRVEReQg3Glbn3rLuYlPd86mbb6pSlJQLNI2AJQP3mr+8L1dHPobejvFIAah6xlo+QiQ0DcYrji4i+ZuSlwjLTgwEHTPK0dJ253tW50HD+0giTR4OUOltGttWjJPaT0llTwoyWXWaWF4O8mVuFTmaIC4GtpgBU5zCuw9o9vU0ByHrG46pQFgICpv8208fnLnz+TD5109J9muQ5bzNXJisedYzicA+B+CnnrifivkF8RK50Zsb4gj+sH8qr9HoleSYCW1nH0QiQAASdHPcvNtiPTQFISp1HgxTXL/jLI3KIOz1NUULFvAZL+iTyRNAArAyJBCl1sgl36+GYeCXyT8sIkNiP6FOY954eDupFPLOlHmm5oDwqEYgRAcvCHqqpu9hQ1eEw9uLJJh8lmRKAAFRwAfAjmzx4xfPZXPrEy4pWzeMAd1+4djCOjsL6HnSuI7rQBJy5a1eBFxHuYfy/vHHJDpISZJIIJBKBraahCKt8F1GEdIhCcE+S91lCIMbjCxUIqlKGVNSHRvI6MflFZZYkFGLqRMXpzIaIiogr8tlOCOhUCQ3dEYwBuuRz83d4vs3rtVzCEnYOWVHXRoAMRTRkv+e5XtDQXWx6yuDOkykxCJiGI/LdxnNsTOSVYujO3ZP4D0UfbQUOkOHUEgO5WYsJfBgeDIh2ZVzCd5YVNVrsEnkqWVcXRUB4yujh70JFNBCLMUqbReJvBT501xXlQv6SQsAQPjzPnf/cCZCMzjCnFdp7JoIb79NsVWG5+OTgkwEp2Wn3dyy1jwR6xkbxa+qVOuXE39tduMYy031GMYzzyXJht0TWaAdufoq83NSHjtv9EUYTVQzdDQRMHhTS9cH4beo2DS04BNGAesKhlpC3XT+oUUSaPUCW1WjiroYg21kfZF/h8w2teKwbrADH6ALZMkH3RIFW4f01uOeSVT3QRjnl1ZYXKkUbZYZNo8ZfxLkzivV3ktlTJyQzD55RN74jPx7aj08mGtMNz25PTWEwaduQ/gVaaKyKeCuG/l3aQy8CsKVykannxEGbKTyJOPNBjDswPXc/Fvlz5KWza3tksiyPk4WwGsiuugD7Z22AHazxsaw0J8vRVFZNfaSO2SbhjoD1E6tt9guEnQT8embqPWnJZpkkAvEjQPpOkGfOknUnGiFjIDdsmKqi+OtMYEmSZ/ZF2lDogT42kcyG55sF8EEQt4FuBysH2drrGaZGm8ZfmOFG48e9nDyxcb7JmfYaThLO3QD8FyDI83pms5svG8SG9evGTuqezTS8pShVkvRZUcve3r6X/er9naym1s9O8Jjgc+dVnqvD/1BjQ4ip6FD9dadhez0rKzM70OFNkw1IaQSE61uInYVnuCcnT7ION0VSSewoEYsfQgxWlmUlowvZyd1zWACTFo+96amZJASFETO4sraBPfbJN+yZnYfZQJBoOQ7aikCp8VC/oVtD8r3PZzvgt+Q+XBfoTXvslKg1nDjrA+zO7wxkUy8YznrkpPEm7q/2sTpfgKkg1twMDyvsm8c/3//WAHbHCx+wxz7fzwZlutkhe729EBQMNwljA3knvF6SOuly2At43jj5J2UQKOvO+UgxlBOUtAxmNNSEcEcJoajDuoHBIkUkYj58n9G3gA2B8BNNumDYAFa44SN25793gUCdrAK8xDsYTeH2zoOm8MkHoB3DEzjBAR7qh2c4104zEoA5129ux3B80XknsZmXnM6cUGjuPlzLHn9rG1u7q5y9CwmTQfr8fpaHTTq5F5t01kmsf0EmWzDmHFb+zNts3d5K1tfpYHW2eSlA2MdEBUQKGcTwwmLTrvTjhcUdQNv7msv6OzMCW+hxgSBhwOjLN23RWS4RgPVoLNvAA9gz9uZ/97GXPtrJspyYtIP9lEeDAKRB1Zafn8UuH96fC0hzLj+L7a/1sZX/PcBOAIFWUWY7JJLoyXahqt3wZhjgCITD/dGubnynKQl1eDNzAOh2X4iNG5DHpl44nBPnNpDh9eveYxt2ljOW4WK9QZzkt/pSfTV7acdh9tbucrYQxNknN43dfNEwtu4fb8PB0gxiSCJexydo8fnCo8bJ2SE1oxqjmo5vk2xBaiOAly+t1Iq51rrCTlDogbDBkJ0wFdIi0R41idKuijo27/UdjGU6uZGX76Q/JKZC3/nrrw6ye8Z+B6NJJ5tw6gC28tN99hq2m0Iw/D3THUpDXV/VxfS++IHeYEbCkT439ivZGwQzGsPB/c13BwNIF6uAlPn7Z99nG/ZUsdMK0lkvp8aCAFwHgfaDcvnsbuns8U/2suc+/JI39zsDe7Jr+uWyr3FByHpni0TNwLAdr9semCOby9s0b55NGmcLhGQjYkXAEsjyCzMxXmeDueRpo1Husd1Jw3PL8tPYWTkedkquh51sfYZme/hz/fDnB9iuQ1W82MDcdLAR7Kzoo80eEppyTWGTToRuRMmB3EwPtXUpju1ycn+TS9KXsLyN7pXNvndiT37yLdu+Yc/uqmSF2S62M6Qf9TaqhXQZpKF5los9VvYNG9gzj+WmuVhOpgdlEVQzuc2P4myGw+HTC5DRZPooSsgsEoEmETBfvkbAxdwaM7or9HK2Gddw4rPYj4++8acWH9JlEuHQITLs0kiSxvLhsDlOrMTIk5KtDEa8RaR+I6OR0cPhV4xMGt4iWV3kOTrsj4skRZDn+f3zWV66m9XCPalk627uZVuFZjYFpg/7u0Nv8kZVA7v06Xd4V0ivku9ysHqzbx3Wn6NOTG1RVCVsKBnm/rn48h6VRf6QCMSKQJoDntoYZJGwYMtkNQuEg+iYIHgiH0uqoXAbfuzeVxlgg/vksN75pmHpnV0H0RWo3UAHdnqEOU2i8aCpdIdmqJm8UzZBnSRPIs9ze2NkC6Ykq/pT5XUsFyK/rxkUqQiFM+mOHrldGldG1+K3dX2wZYdErcRdgmEVkkWedmiXbEOqI+AyFAdiKLno9rJlolsfSYWAw9xO9jmEGt5Wep6x7yy4Fl40uAe77nsns16wWXz09WG2+MNdjMGX2z4GX7MP9NfkSzXLoSt6NiwZR4508JbKbwCF5UPqpBQgSx10ni4HZke1cHNwzkV+kYeGAi1kp6o7LGEaHU2qkEkikBAEfOFwJsaNYCbc8dY7OiEVJ6gSizvZiCG92Y4p+dzFUDybNNB0OhysW4abG4ZfLNvNLl/7b3L+ZP1hz6jEUJ6eZRsltJhab+Q44Epgm7Y1XncQJWbjcLxCpAOBqK9hGE7K45YSHaa57jRNswFvNSzBZg9dRGSjTehtg3lk0+R2aiJgqAZUnva71QWaIqwo2SLIANxksuS30/rksydHDWM3vraN7fGHWD64INjKc99kfe25kzwMwZuQn1kNDeDtkAgj3hIYhSCd8SZ5XLC9Q3z3Q09CTvHNyciUmy7L1z4MYFCeYZjfA2p00qfYo3e8O/wPVOV1R37JLYlA2xDIcWi1FUG66ZHMx6ZtFSaqtNUWRBPjNW74bDd74I1trDeG7SFrHx3QQJDZmWnsspN7s1FD+7EffXcIJFEP+/7T77JMzFCqsFWn+GuKKKXGVuRJQMJfjacDdbR4GgGrsf4Q6Q9X1bMMEimbuDmoCHXkIET8W87oD71JBvt8XxV74PN9rCd0oESgtkm4aTRVJZWsTBKBhCCg+30BRXVAQFMgP9joZhePq9WkcswWXLP9MIaH5BmJnaKp9ADj9z1v72ArrzqVTblgGBsxtD+bMfxrtmzrXjYQ0ioN322TIMQh1E+1iuFxrbU0uy1aR5Zz5lLZW3squd9+Ht5AP+mbw+rhs5lFjW4CQfKx1ehiQBl9/UWF7HcXD2fnDIA3EMqk20SqNu8UfpfoRjiI9UwpzTO/5F+JQBsQOOyCAwczfHYZQUZ2he54MfRzIvY6EeepFPwDM4dOgHs5/2D7ZAhILNPFnvpoF6tELAtMDmTf7p0H47EOX+3IGjt6m1t8qVO1tLhRreULYIsmcp9NSJtvYsZQZb2PK5GvKhyAtiJuJ5pMEmYkgdJYhWJ77od+ZApcHYhsaQL5mzvh6oDhQGt6UhRPXjLvJKzn7qpM3knlmTotAnPn8kdBC7lowaIqizwjHw97dN1qEffshARJtgiachn5qSYfVTzHBrQPfgQJoaSQdZ722apH0HZi9SIYtqsxJVbdZ/i4Cs4WBmoaYg+CvnLzvhr25pf7OYhnn9CT3XxaP7atysd6wfGL4v5RIGQKVdcPAAdI3QMl7nXnDGJpkFq/qapjb2HYzkDCCATH6+jwP9QMgI5U5VADGLvIJBFIDAJV/vo6SHdfKCTZ2eV+p64d8+jxaZp4ZjMhRFD83W7Wh7bpWcYkeDYY8SnySQBC2o/nmJgzbAuxjjeJ/vCYnrqqfOFQNLYLb4RDmGXUDe79/BFvzNYBG9QAPukb+s373vovO3dQL1aQ5WazLzuDuyvNh16Ee87yGQnITE63EPv/f3tfAidFcf1f1d0zs/eyHBKvqKiJQDAar8QDdhWPYIAF/rs/E3OYmIAaBSXKpbiDioAawcULjfEX8zHG3R8gXolBLtGIovHc9QKy3sjC3sfM9HT3//tqunZ7Z3dh74Ot+nxmuru6uqr6ddW33nv16tXjE8awM4+Fcyhc/vW1j9l7ANoREAvIL2AfCSA6prwcZ4ee5uo8iXMIBvtI9VQ1+h0FhEoKXEOQR/jSdSVMI7ks0mcaPInr3spEqK9WR9jb1Ce97CS9B66/PTSZXTMWvixgWbO7op79eSekR/Thqr7Sh8nJtAPbBvLNaztfGnbI+pIFtAoonIfCcYXgrHu7EVWDkMfCxmvD15VsxcZ3GXlZyYDSeMGFJ7PTvj2UvVuyh5VUh7GCSGeHpiexsd89nJ1yzDBR7fXvf87mv/bfmD/AvkJ0qhnpmOGF2raiO7+aMSmm8+wz+tje/uKq/A5SAIPvZoMFhXbqM07LrJviVQez7fxjEjSJaZSgMgo6zD9NOZklgMsEjDYEMmUaBheSpxw5jA3FOnfCVWKA3t1TzY5BPDFAfYL5pIppYO/NcLWhaZ8bFcEpFYOXrttraxptPdwnAgkfNHN+LADzNvj1q4L50VUYkb77rUFs0klHi18YyswAzb67gZTMT7/9X/arVz5mBkDVnZGRt3v/SC0IlmGaGS4RlYEnHNrxsPcrpmpwcFDA+Qydml5Fg+6TOKTGztELLygKB55QLZICMdvO7x2RwejXWoBdPPtyXy17dNuHbOGbn7Ej0P/L+wpwikoDPTWAjhUu03j9Z8Tn4wX5f0DsH8ZetVdp3kBXAtA9LoDmf/A1y/+0jN0JvecZRw1jKcmJLB2iek3YZJXV9cID9aPvfc7W7tqHez6WApG+jgaJhtx6+cRl90Xj5rxY1GY0nNgW9nK9VPEHAQUyBRPn03zvmqZZjjWQGRBvKK5XOzIpAEmNGUAf3lpUwr7KSIHxS4zflBWTvBpdfwXTxFd3V7E1X5Szb8rrAJzo3wS+vf0iTVoIXE6Du3cs/mXpnNzdAjyxJ+5mkPqqPsIcN1SXhBDiQI9O8rM6EP6GV3Yw9uou4c8zHXqRSpgxwIZJmCTR3ifHpQVYJQaHPgWc9Da0GBaWwHA4XWrY+vbYC8ac2MbO1b+iQAcpEASXiVB6dO17GbsSvkY7y3Cc3le/0Xw5LWrR0X9/s+HDWB/dn5qK5jAgMSZgsngETJbkXIUE2g5Sp4sfAx+Nrgw1w1bKOAaejvYeDG3rEJskOnovj1reNybuUaxvBRWPwcSRgaZShSWbpVjzTrtppgM0NYxSVcBR2naDQp/hOEVt8Ef6TjFimZ/tXTDpYxHduOOhTKWOigIdoQBNOmosN9dyljyzDR17FPowekvvwo4snXrkCPTbAICTzmV8/IuSiSIti6F9jyRwxqfp/Wu39g7bQnUROJMaSSnBq72D7YcpLsZb01kfCqQcJP3HXhAXZgJsGK1/x3Evvsg3GGhpLXvfDiA1118WdaTGroKiQFdRwN0EDszbFtoaFw2tT3UGAsOvITnuxo+OLf2I8SE/n31uHXvjNyJrGc0J1+N1tI8oWmMFBXpJMAsLe7Rt4Dwbk/bRM2oVRGCyB6UjXff5QPoQ8gFoOy+KusodD/t8xVUF+wUFXN05rGXegM12NTo5IWi/6Br9gr6xSlrcnwCI1LZVzv8JdIcEnsXFgsiYgC8Aqpq4qwjflV8Uc3PcB8t9M/KhzbVXRNYuzbuyGJXXAKZAYS6kRYeXLcjGZCTfwn3CyFxZcnRtk9AdEwu5GH9GZAumU2PBPAGeZfWpb4Dv/A98v9E9NWoJCnXBH1ESjAAcCWzAiFVOnD50VH1SNdIFb6uy6B0KOKygUKiCYDnztNgZEUtJeqcqB2WpEMyhFLHNvVqAxcAzJ8cmRRwITx06iybIVnNDgKfq3F3RBshEiXyWRKAKd7T/E1kW56iBqStoq/JoSgFXmjH87HnHiuyl1WxIoNpaUyp19MqOzQfxf+6bPfFLZIKVpljYLnJzO7Sla0859XVlWKpJs/AKQDtK6obnHEF0QOjLJ4VTYpNFeapBN5BHnXQdBUiawSw7dW6wSYV9efK36166B3KKGQlAZIfTKps/IUosKBC4GQPPYGzv56o5Ez/hmrPaJbwatTrzbYRhPHTKtNieO6s2E2dPHH6MG+1MzupZRYGWKZAbE92hI3qEhUPo7WriqGVCtSMWxj1iosh2XilfMPF58WQu6ZhdUyVvVo5m3O+EYHmuJo68ZGn/OTwHYKKIw4nAO+mhspjInpujuPn2U1I90VYK0HJfmMFBt/4mhu111OkRVJtrK/1aTBczjAce3i9uEwPkqkNinCfFEuHB9lfMufhtJCxwCa9m7ATFOvIHYR0mDNA83VcS/HVIcJ19zP6uI2+lnunjFJBmcNxZ6UTqqLKQdpSqqINfDZYyCfCiFPnPkCR9tcgDE0Uyr0bwpJhFIDOC7rB8FqmvB9tPuk8lvhNR2hdAdNpAKfy+nhqO6UmUeVL7KKhSd4wCrkhZPmcyzOK0/+OBJOrBignqGDXhQRIrq20nf8fMCeF4tVtT8CTdZ9DR9s2ftB3AeS/3990VRx2jRY89pTl2FAuKWLD097k1MWsGZZ7UY9Qf2AVhuWZsIhgmxrc44boqSECKCWp/m7B4IFHjUfP5fQsm/0U87g5MMqum4CliF4l/pz56O0xsPmEGVurjK8gH1PGAFIhC5cG5ZT9ZNmcSsfoc644bWP0DPq0SKAp0lgLEBEE3Vz4v+z2sdV+mZt7bTVAsB8RkW7i+ztCtG8XT5EIyTgpvDp5k8hDcJPx8wio0KCbsaHmhmiVuwxfA1LpuGI4ZrtAd/WbxQJCcNCjVRxuIp5J0JQVc3Vyyf+hyrBx8F1IkdX7FBLWNxjZPgLoD0nfpnClvC8mxBd+7zcGTMg9mCiJj5PobM0MFxL5iMkkRfn+El4MLmchynrd3wcUfx8R1cAEqKAr0NAWoPYL7/GL2mdhvzfmDY2J7DmVB05avQPMVhhOqeZsn6YvFA63MV7QMnjQr7Hr+GW5bV7FIqAju2UlvogC0NfJjcOEJyZoTCT9RMXdivkimTJNao5aK7wkKwE0d6T8r52e/CPHnJiW+H5DoNpZhwiA+jJ2A7N+WzZxQFWOAWp6vaAU8UQiJ7xi5PlwwdR9n2nRM15uUMSRQxUk1/wZitIKO5AM7NekacVso7TEIqaAo0JsUcFe0lc+bRHMYT0McJfGdlmKr0JQC1FfhvRnk0fVrK+dPeZPUl+QntWmyxqvWwZPS0IMA0LJ5P/m333Guj617F37rFChIGtJkGo1W0UgtKPOb6qvH74uNVkpclyRSx16kAInv7uy7Tw/PgP7zE0xokhSpALTpZ7F4YjLn0cjDFXN+8mfhUDqY1Spw0qP7B09K4YqepQuy83kk9Eew/pAAYPykDG+JOuDCYZBkAz8d59dlcydtE8C5n9GKHlJBUaBHKeDOvtO+O2ivOVinvQdWNEoN1/gRopjXMVio9rnyUNpVInrRogNO9B4YPD36z7L5k68Hh/UIDyRjAgnAISdJGisxkM5IfaHRjpjQw18F2hSKEV4B50BqA/3nXV0psmLexHd0h/+UWWYtxFM1Aw8OHKoMA/M6LxmpaZcI73JkltQGt5FtAE+0D2G+FBRpK+ZN+i1Y/9VgceGkktyjD0gzHAGctIiAR6055fN+8oCYYAsOSFr0HwAZ6DUlAA0GDSyC2QiB6WeObUcM2HGjYw9MNRxnJnDMAJ69FTX4tNLfZ8UWtLRgltRS0yHWtO3B3Wv8lFVv+HZVfP24Y/hzgNhyAqltQNz20vpmSrFgAKI6Bm1uReeWz5t8BypKqgw6DMxG2De/lKpVaxTAPAbNZxyy5Olpezh/EGLkkHQwQmi8xIke/IEkZrKOgajuROrfwE7sk/fOnfxVe1Vu7QM8QmQA6JszTjXL507MhQ3ofeC+NMit+A0IM6YoNr1HA4OfTsu6goAzSCZdYvM5BZwHf687SN4QwJkJDnTP/EmrL7H3LB8T8PFa3QdX6QPClht+T+HxLCEZOs76FxxHv6AjwEktoX3gSU8QgLqzdxXzJl+tRSM3kd4vBioH6QxeTLdLyy4NZlnlBtOyYfqxisgB8LSDMeUyXaqgKNCnKQBHF7wAnOdVo0YJKenEyK4h46re+wo60I+dQDIBKEmSB6sEZRGjB2cp2AWz9rHySNpPGrbGIZVGOwPNuLU/CAci4LgAHPvmTlqcseyZEnyUfOYLDGZmmCpBoNw+lUD7a9FDT6Ax0doMmo0Lh973MeOXpfMmvAVuk9++ZMnQhISEmtmzZ9e773uwNroeorUqpjspgIFew1p3AscGoKjxJd11qvneMl/idzl2kfhfuGC7iEUjgE/BhR4cYrwrpgsLg6hpQVQPgvG7TdDaVWF0hO6dAzgACCOuCyB66LI1I8OO8aCTkDQWzpT7P/FpIoxceekw6QCVIKY/7ASicyqum1KB9zXwiy5atOhVDBqDQfjJwWDwQxrVERSAdqQlqme6lQJonxp+9u233z7MNM3T0VbX4xoo6Q0Oz1j6zHxooRYxH1YUmiGALFRy/ZkRInUiPOtCTOfgNnfqjnPFPqy4Em9NEjQxgh0M7RfbvQURUOCDkCX+13OnfnBoKHS+Ha5fQqMWRFwatVCxfrkiiQjOhQmDFS3llvmL8nkTpwvgxEjl5OXJkfvBxMTE7wAwHyOyEHDm5OTo1FC9ZFLnigK9SQEa1Ak48TspEolsx3UB2u1QqlNDe3VXxNFKJB/Xz9PMcDFPSMGsqNhIjsTd/sYUCA6bG34dS8s5C9U9EdATzhTASdwmMX6dAE6iXec4T8pBBg/7m3H7urMcjd+Cip9L1kwMrDJAlPi3vgsq1DhEZcE9+hM1GBJbmAV7zDLZrZULJ/1X1D8Y47LlK9NxyZIlp0Sj0SELFy78lzeeGmVhYaEEWe8tda4o0CMUoDZIBY0ePdoh8LzllluW27Y9E4N8bl5e3upmkhIBCu2DhHmNEUvXp5c5oT8wjvT+xHSY81BW1J6pD3cdblCuXRsINGmNuoFtcBiWpBYlcDu4e252bCscD051ttiuJQIRn7zRE6LjfPDSZy+3uT0XCtrjnChWg0XD9GKCRSPOrrOV76LnaUQlg39NrJ4iFadpvozhKm/f/AkbRRkxglPdm4y+xGFSo3TrQe/j3HrrrcdblmUivsSNVwdFgR6nQDNgRA3QJonb9OP41X4r5AGY4Xc++z3TsoM259PIsUhsG22hD+07IBpjfKhvwu27oYt6hmr3oU/nayFjRVkQDj4odFJMF3l4/roHwDzET739xSG6XnsV9qC7DGZNIwRzZwpVC62tpQ/QS9woscQaGfkbZOzOCNxtaxs45gcq6lL+JlYaCPE7D0RvXS9Cozt+MJuLzdahYa7DO12E0f2XGN2f9NBanSoK9CgF0BYzNU07bvjw4X+ZMWOGKQtHvHfQl9FNj9T2aT8kt12nLVl3EdjYWWCPLgQzxAWI2liXTIFMFYlx6vlAgEl1AK/j4+R7AxxyKTC0AEarK6rmT94hqtQK89PZ6nbnC3MYnWqS+OlLns2AomGaw50rwXT+gAcSwOGF8eqWl6PrztGsaTnQg3DDYLRTKDw9/wsj9cMVkUnPNgClZwBoC5HlSI+GeTqA80lckyXDaFxXxYtPbclPpVEUaCcFRF9GuwOWcRLTZycnJ/+xtrb2Y+RzGrVD/DQM6DSr2USC2m85cdzaYUufOTvEnMshn2VDnB8E0Ixxo2JuQ+RL9eguhgj1JqNqtxyuwewIO4SC8cEutTvwXo84TvTvFfOnlIh3okU9BdiwrT3vu19iNL3ZneAZKyluBPtx/vOB1+vNiy2HXQw16IXQSxxOu0yKj0BgSiYSVKtGsT5WR7gswr391zc2Qy7fkAiNcxCanuI62Hm/uAdCA7TN93Gxnjt8dfkC2izLDQSa5IW7YwSnkqjhDsaIf/jNN9/8Hq5FnMyeGjB+UtSX0eqoKNBhCnjblBzEYQnyGDI8zO/3T58/f/4uGd+RQih/NooZrJhFZdsdsuzZkYDNXNt2zkcXOwtAKrKO9a2oy6hQf6XmLwJ4FBwb+7UbHXegfkf4GAvuiduPwWGKPkzb0cdAuxJ5rtcc7YVouH5NVTC3TDxGoDm6ODaZLXPqhmPDm3VD3k2zjANRujl0+T8PtUPhs0HiqSDqiSD0CJaYlABuFLQB/WnBGB3FD9TCviACD+kTNA14D+QiZgaBsjRBSHNTOBUbgNLdcD0RtgT4+hrguVBL9b0pnJ3KfASnmWMHnUW8qLCIF+Z2bLLH25DlOY5jAaaZKSkp98ImtEzGy6LVUVGgqymANka6TWmKRP08vs+0qcigg8GeNw72mbCs2ZyX2Tj7DmZoaL05xrR5NnrcuQDSo5gROEwsnLHAEcb6Lvqy249pfiFWl7j6UJ9FH4ZZkei4bh8mjYDozzpE8lAtAetO/H2sc/6U5egbYOS+q+FFqA8X5wA0W1ezNaTtghMiak+HmDhPru09HNjwO19INu3wGBBuJBwWnA7Z4hRQ8nBAZgoqmICP4ediJRPRNkZfUXEapejDQP3iYGocQFsPHA3h2Wp8ip34Hq8DQt9yDF5Ufv3FRU04ygMQmxrOKABpcXFstrKthKJRHiM/HwWdEelCcT4jNTX1werq6s9x7yI06mL8tKKiIlrtAV28EEPamr1KpyggKCC5SbSl0xGxEr/NOJ9LbQs/AinmPafr9oTgpkwjmLU5mv9SzjDbis5mlvbEteNXvyvypIw8OlGZb/oda481bP17FndOZI52BjrmCQDEQbhPrGlSA+dI0ib1YwrUh0U/hvqS+nHUpPkQmt6njef3ADzfRj9+DUmKLB97v/r6iXsR3xg6Jy025tPOs94Az8YqEjdaBKX06OajRQ4IUlw6LPHTmvIjfY5xNOToI0DZDMBMEoiYiHVkAaCODZwCUAIwuVYLYNwHbvXTBBYtSfjWoN0ll2WGm4AllUw6nKJClOkBb+JpgbZ3vnBBcsCfnA272o+uyXr6jcaK0mONQJqXFwS2t3kkF6M+GtwlAMkr8LsB4jzZ2uE0BpqkE1VmTV5qq/PWKEDARYMy3aeBGdYdx8D8qMgwjESYzP0WOs1HKI2r26RkcRweRe0/oH/x3MIcjaSv/A3Zx1qcr0vLCIyuqwxNuCbzqX/kFKC9SsmMVGlk3kShBW9Eo4NF/m8SdwxFRz0aFTnasbVD4FYn2bbsJB9M18mmyHJszDtoIYdbdZrDqhzu+xJ9vYQFUj6tqGQ1YvLWW+WmuEHv1+539GbX0fPeBU9vrekjFOIjFA+DqRPEAtJVdkUgQrNxAMxShxVCl9lKvtRgCBCXb8y+BZ/iJhT9OiSHjcC3jeGI9u4NF67d461OMMgwugsRxBvd5nMCTNjfHX/ooYd+hpnQOi+YtjkTlXCgUUAMxN6XnjNnTiomhn6G9vMqQPNd3GuWxpv+QOdeMX3Fi1POp14JFWM6sv3lrMw1fz1A/rTaEOWjv7FMLJ7pQvGZuEsKcRKriOulv74Dns0JACgDfgqnG3kQEcAtFhfTh2mekmJGARwpkM4jD/An3kz8xeLFzdb/JHhSiuUvTtmaOth/drgeOhvArRmxdoI13Q79yybuGBvKto78LxqqEItaz7HpHeIG6BnJZeL8MKD1h0j1FRr+dFy/5AIow7lYEdI0B3U1kCkgB1cYup8GTnMiaPEg2kkTe02ZpqN0kmI6PX/PxqlzsFZkWbjO2o057Skzs9Zsc/OlTtWmPoV0SIuklJr6scsxCwbJzazpYQupAsDkIJYkQ1gGuP24reU1za6brwS6dHMZ/Sb7AogjuRBHVrw49TSH21tRcRiAMsvwa7rPpxOIYpLeJoPb97Gf/UOzslb/xQu6bXhRSW+HPNsUFxfPwDOkq3oWHYHWx8frqqjR9MmG04Z3VUk6QQECQojlGq0OIt04qXWgO/99IBC4NxwOm7j/A7SX96kd4V7DKqKOFEltuBBiOrX9+zblpESc6IPpQwKXVu0NvwTgvBTA+YXbN4hhUO3RJTLZIqrgUoAaj2gk4wu337Mx+25/km9+qNZ0sDDKjkZsG8iHyXstTdP5mdEQvEghLFpEYkqwrQ1KphMTSXj8fnSANW7xDOe0/pjsqVJwjJlddFIMk3mrY/+hgIeDtAgYpZ0w4vVQKPQs3mQ+2sf7bjphqE7pOhJITOdiNh1Mw8ZpZ0Rs87HkdP93qveF767Qvj83mBWMEkeam1VIkzgqeCigwNNDDDqlmXVovrHsyHd7uDZ6sc+vnwiOkxoo0cqCoptcxs64dvxTT65adYpvxoxgw8oNer6NQYAodQp0AmzKxbCeoABcb64Fk6bpEMsWI34FnDfcNXfu3GrcJjFecK04tktd0Mb6qGR9hwJiIjE/Pz9QUVFxKdrDjptuuuklfHeSSmjAFoM2VVdOOHa06kJM50EBivdsmHqNZjj5MIEO11dF/mfWuU8VMLYW7RLSmALOFkmswDOOLAROboOpWb4he17UtJ9HEiyOYpYO8R2TSvBz4kCBztiMGW+alDYnpxAT/+0XZ0gUo05BeQFIJSi+issPYdp0c01NzX9wvg5pCGRFI5c6U3pGhf5PAeIe0Q404hylAw+aQS8vL38uPT19ZGVl5TK85Uv0prKtuOeyvdBlu0IsnyAmsTdHH9iQc3hYs+5NyfBnV+0LvWZo+mXXZK3+UJaVmxsUnG27ChggiQU3M0DetV2vSY0HP3vFhikP+BP1KwCizIo6n2Cy/qu0jIRx1eXhl8GBTr/u/LUfUMYCcKX5RrtKakwsxTUCyDFjxpyE1SE75s2bV0kpbrvttqNgimKiTnKSgL6dVAM0ZqLO+i0FZJsDeI6E9EE7FZDno1WID+G8S753gQNOkscWgCzfNC18wAAgAAAUR0lEQVSb2/YjgWRjcKTOXFLOT7pZiukErP2WkD1UcQWerRBammz86d85g2vqzXd0n3aEGXUyZ5+7dss9G6bMAy+6xLHEdNEfZmV9/x7SGxGAttegPr542YE88aLTYLJgMzrSOIhqwb179y5duXJlmMAW8VwZ23uo1YdP5ffCNyauUXxXnA+CaH4FwJJWBN3SSvU7DZzUnuHxjFHZyzdlD2I2ux26zStrKyMlhsEvv2bc2o1UdlcwAa28w0EXTR9FhVYoIBvSPZum/Ap7Rh157XlP3SaT5q+fdrKlWavShyaeVrk3BFGbX33deWtIzO50A6RORjOtEOVgWywMUGm9/AXIegnAMxUG0eNuvPHGr2VdPMdOdzJPXuq0CyngVbfgWwqpBsc0fM9/QzwfDf3mP3H9YypSgmxXDYqyHVPeMHq/AHL4Q6mDA0dVl4ceD/ii11x1znPlZPg+up0r6Si/gRwUeLb96wtgooY/alQRwK3QWvXGdF+ounQWnCMs1aEsghnTrVErddkNF/61ljBvEayEveuC215Us5QNZcNMZQgcMMPtluAiEsC1XI3O9jV0pGu8eylRPd00HdaNNauFiugUBfBN/PDknoGtML6hjGhSCLrN2QDQT0844YQnacIQ0V02ABK3mQdLENLH37Vp4lDDNm7FppFXhOqwO4LNZs4av/bvVA+vfSddq9A2CijwbBudYqnIZNedGPKO5vdsmXa8E7VWpA5OmFBVFtqp6/p1MzNXP0MPUbrOivKUTyucy2B0vM2DBw8eU1ZWtn7kyJE/djsgPSICgSh+CkAlQbr5KAet+OWRUK/8CoPcTSjexKaBP5K67LjqdAlweu02KX/MpE+BK8j7IaZ/q6Y8/BdLt66/PusZsT5ctY+4L9COSwWebSdWs4YtuMvNmbpUri/fNPUyLEm6MznNP7Su0izkhnbjrHGrP6EiBNhiVl6Cb9uLbUwpxTkXDEV9cE4AejlSkYi/nO5Rh0AYAz3a1zhvsqyU8qAc8YyabGokbbee4RskgN4bQfvjcVyKQW6F5DJxr0t11t5BffnWaSN41F6clOq7pK4y8hXWj18967w1a+llXW6TOF3VDjr49RV4dpBw3se8DZY80FhRc6E/YFwTCUXJN+nCpNpD7pkx8SHyENNpfai33PhzAk38bMzMH4mZ+dfRUYehw96OuJvj0+JafnvVeVogzoGi3BVipIsmrl4MZPQMrk8E3a8A3Wlb6t9JDpN2rYTIHsZ9WqHW5YF0lgWuyRypk+oqS2ehkMWBRN0fDpn3+Wwt7+rxa/d1sTqpy9+jP2UoO1B/qnNfrSvcy8WWuFEF796QfTpk/DtSM/zjqsrDnxoav3Fm1trH6R46UIPelK47E4iTJDtBygPcDHVksfTzo48+moy9lGbjeifKu4zi8SOTp8OTkpLqya8oXSOIjh+r0yi43yuWgBC7q/7F92qLB3bQEG7N2VbQ99T6+vrnAJ4/pUUO9I0QBP296peuIG3su8V08JTfPS9OuQhL4e5KGxwYXVUWfkXX+PVyXbp3kO+Ksgd6Hgo8u7gFCJMQ5CknimBofxmIvDglI3BYdUVki+bYC2adt+7fVCw1fDgxaUjbxVVplp27Z/druHE0fnehUy9AHcieTwAojjIc6FqmO9iPYmUXaCR1xoIuBIZw0JGJ489BgCg8Y10t9whCPHxYsijcDr7pEkc8I741Ijx5ubc7dojXa969adJx3NGDSSnGpXXVZi13tDnXjl9zP+XuNVPqWGnqqZYoQB9WhW6ggHeUX7o+Jz3BsK6Bk+eFgUTDD29NTzhcXzz73NVFVHR3mInI5Z7uq3k7Pbkvo+1nv4Fz5p/eddddtZQGnf5HiIevVGc7OjiJluIZ93nq9NJH5MEo5hNI6vCT6Xg474b3X7p0aTrWlNMCBaF6gbVDBkTwd2EydgTUI6vHjh17SVZWVktG5Q15SDp2+ohJywLXiQfltWrTT4fWWfU34GvN8QV0Bh8MqwIJxq1XnlkIn5jdqyai/AdyoI+rQndRIK6hr9yScwx2Jb7R8GmXC+f3UedBLcG5a+ZZT+2kKhDgFufAa71n24NOVk1+3wMCHmaDt2VkZJwB05mdANBzARSfEWDiJ7kuURXcEyIoHaWBvlxW2Mm6dvvj9D7kSJiWQpINLQokujQDOPned9xxx7fq6ur+iDQT8NIvQHS/hCpJ93H4AawqahcuXChWmOFa5OPeozSU9wHpjjRtCvGc5qObLkuoZBW/ZfCamZTuH1JbFXnB4PpNWFopnHiLwbuTE5RtqtgATiQ71wAmQfe/OnUoaRtKpf1xy7STtah1c2KKL7u+xoSjbr7Mx42Vv88qdJ2EdI15U0tvRnWJ09/JTv8dpP9/+A3BbwnSCVMWcKSzMWv/C9RxKwDzbsSXUB74NQFVPNMAQqTXA6DSLLIDkKIZXRk48hDnyK8lYGnIQz7gHluMJwCn+/F5Uf08ICnLb5aHfA+XkxyPrE5BnuQe8GXKF8cTkPdTiCuH9cIDEMUfo+Lw89Y9/poe7boQNwATiOZvmpaDvYGCqUMCI6v3hYrh62v+teeueZoKjVcbdV1FVE7xFBCNLz5SXXcPBeJ1Tys2TB2LfnhTYqpxfn2VWQcUWAn/ifeS/0SqATpvE9Dtnlq1nKuc2AB3+TukuBG/w/CbBuAV9qsELLieid8X4MDWEgdGYBYPZLh/wOA+R+losqoJMLvXFE+iNYGWF7gOmLebQACcq/P9Aeo4HCL3PxYsWCAWG+Adr8Tkzv0YJBgWIcxCOfluuWT2lYSfENe9hbkDRPdNrsWBJrWdjM3v5Fq2My813f/9GjI94tpiLVT/yMwJ/8C2sxDnPROW3rqq8+6hgALP7qHrfnNtsgcMUtKSOcyQzgcnmllfHQ1h09CVXHfuvy7rqRLKSIpsHfXeRHkcKBCAAUSE3k/O2nvBkETY7du3l0KXKjasg/OK8QCbpwA6ydAHPgxQnU5l3HnnnckQdRcDoLB5n0O2jQ8DfIQ+cPr06T6spElFMprtr3fr1BrndsB4yi8tLc0v9baUH8oycLgI5Z+J4+ewqXwIQCfqDC76r1iJ9XPMhNMSyAuwxHW9+wwNDD/AbweeJ+/+zYIcTJrd6OKIILZ3GTUqR6xgo6zJHjPDGTQVI8b8lEGBk+CQZg/X+dKwYf1p7tlPk7vCbjV/o/xVaJkCCjxbpktPxDbjFFZsxp4xtrMgKdWfWVcdIcB5GELiQwDRt2WFSJfVnSAqy5FHgEkTTlDG03H58uWDamtrv+Pz+b6ASc5XFIdjOvyQrkHcuZhM+QCeoc7CvuHldA95LUb8AtM09wLcLgPgPufGj8BxIX5D8XsG6R6ieArEFQLozkP6veAWF8k1/Yi/EPEPIslQiNSLIVIvpfS4fyTK3I714sPhzu01gPv5rk9UmhQ7C4A/BunfAoi/QYbqyANZt6hCoOzEOnPcF6ciopv+5ABJy35lEbQyyHbsm7EO/SSsDNoHZnRpwJ/4yFXn/E3Qs6fbgqyXOsYoQKO0Cr1DAcftKA0gem3mWuKE1i9/cdp4gOZMbP9xJWxFr1yxcco/sXn1ioqsMetzecy/InWcLp5capEKADLSbcab7Ii01113XQVOXvc8yJctW1aJZwjYDsXPJOCUAAUQ2gjgPBzpExEnRGZ6FmCWDlA7HadH47cHz//JLZfAazziJuC3G8+uwFE4REFeIdyjGeWPcC64RSoHoPolATdA3Y/j5xI4kY4DYF/BkX4NAc/isRjXjUgS05vocul+Q+JuOBHfEQ45pDf3VW9MTArX6JPhc3t2YppxKrweldaUR+YZDv8TGblTFeQzytdmN3yQdmSpOM92EKu7k1Kn8HIecB12EjR809G3f52c5ksAN/oO9FwPhKL63+edXyj8fJIujPaW70luVNLBBR3e0gSUTHOAo2x/QteJtLT9CJlJUaB7Arjc1TnluEfceMMzlMgTGtJ74pqc4nkxkeQxR2pyv6cuJJfp/Wb5m6YeEbWdn+MlrkgZ5D+qpiK8F5zmH7nmPAjJgwYpAZreZ3qqvqqclikgG2LLd1Vsj1OgpY51N7x969z6jc3sK1PSA4eiY32DGdb7fAH22NVnrflUVlJyJAAJAp1u5ZhkmS0dJajSPdSlgZMj8JLpOwG4IgtvGS2VQ7PtvQ2S8l3do+DevVYXFL/ipWkn2qb1O43x3yan+xJgclQMbnelFTYKZl9UKFaBeb5rAy3j8laXvUABBZ69QPS2FhnfafKf/3GanRiYBICdlZTiP7W+JmICIh+H4PtoxT59WzC3MCLzJo/hOaxj24PIPHr4KMyYABxUrBf4pcqgVweEjtKCBsNFHucxlA99m90vmefZUTYLTrbJhpR2Zv2HjgExkDb0XzNOfcgU6Ug1o3xsdpT03f6cAs9uJ3HnC4ifgaUcsbvneRDnfw8hdgpm6RnsRT9EP32M+fiT152zepcsVYr1qhNKinT/UUoP8TppWFUca3Pnf3D/Z0lpgdHwdBQGcD6K7/awdKRNteuHA1/3E7UPlqDAsw9+lNaqRGJvM7Fv46TvYn86dEjnUszSfwfcKJk2PcM09uekVAtczDMNNopiGShWMEkHua2Vo+LbTwECTHJ+zTZv1qSLQsplFZbmhozoj2FF8QuY9E+AxMCgu96lce1//Y7+5yvPiy2jpOdp73Sl02w/7XvrCQWevUX5zpUrZ+hJByZEXIj0ASvRfw5ztMswPzw5Kc2XUlcd3u3Y/HkYsT9RF9W2y0kmKpq4WTYuU2NbMpvNMHeuagPnaQGYEMnjabhpU9B4z3r/R7ZmXYpBbSqcEA/DrDlNhK3TdO1Ry69vm31mobBzbWlAHDgU7N9vqsCzf38/gGBzbvS+TTnfijrmJNvhlwJbxyal+Vl9tbkbnf1Z7KFcGEi3XvZypAQCucT1gBaK82m9QQiwBHdJ1g3xahAavJyA/xSbaWRWlQ3vRqPDIYvZUWcTbLH+DNB8YebYwgbzrJg+uxBWBkxNArVO8j59R4Fnn/48ba8cdWwS++L1bPe/OG1EWLOzsWVytq5p5/jheae+zvwa9qNPwwD7Be7Tt103dk2TzeSoY1PJBBB5eWIPHLr0TuLQ9cEfQNPgIsaLsOInByOL3LJXvnj+tkvTnLqas2GTeSFMyCb7E/SjNJ2zcJ35Eb7H33XHKZg5fl2xTC/1z2qAkhTp30cFnv37+7VY+5a4UUpIey3ZtnUxeJ1cXJ6G7UKM2qowXNLxN7Cu/mnGjY1JqaGPvVypWwDHMkF9VOkw5yDv+BwAJ/SW8aK4Swd235ZLjoxEQ2dzjU+BHnMc9MyHYLIOI4vzBtQl6zBx/g9+5JHvzzx+Ja03F4EmgIoXwVuWx2xL3lPH/ksBBZ7999u1peYx28I8eBrnjcv+6EHhHs82z3IsfhG40vP9CcYhAAQWqTN3YmJjC2adXoWnp1ftUN0O1/FEQ3lSfC0i43zEErfbryahBEeJvYOwC6qsv9zTvOElcUKc4vBXiw4NhaJnMIefw5l9Oh79YVKqocHhcBiIuRmz5avB0W+FK7gma+I9Zmb90sTKSwd13jIFFHi2TJeDLxa9PsfVa+bG+XnEnjdJdeV7Tmc6uwDifBYk9B8mJPtYBDo7K+p8geutEEtfBof65tCkhA9+/sPH5SqgJnQijpeN26wRh0qASoBEYj8lEg0t1tp6QvwXe5zKghYtAjeZxxjpKqku3lVcdO0NKzbmfJc50RMR9yM8fx4yGhNINmCDyiCOR0swqLyiaex5x07acO34J8QWwvJ5smYgMD7IuXP5ugP+GGvOA54MA48AxFUJs5rMzbRJncQZQYh7N0w9ytKckbbFx3Juj8PNUbAlHYR96VkkYpdCPC0Bd/ofcF1vcVt7j2v2Z6y+vjSeQ22JqjTLT16DioeVirZHQCvTFeJkNIEuhUXiv+lfXuOlBEKZD91pj1qBvPsnaSY5IjnW4vDjabNTAIrfg+3s8dh3SoPLN2yE6uyCGP4WVnOtB4leKWdGSTCrsKaxFuBO4fWIZcJioescWHuzV+d9mAIKPPvwx+mpqkkxfBRrLt5THcSOoJYzWnOip0N8Pc1mzkngJUf4Aprm82uwWzSx5px/hKQ7sRZ7F4BoF8yjSuCw979I+0VS6iG1ctVMj77T5szAcH9GWijEjwASfhtqieOBzOSH9FgMF8fpBj8ckzwsEgaHbdrQ/TKa3HnF0fhrTLOLTrYrP8jK2izc6cl6x036UHQD+Ms06jgwKKDAc2B85za/ZQOQkjlOK7pMmgD5cqt1iBa1R4Fr+x63+MnQm45EISN0QxtKgARxn0VNIfZHwKGS6P85VAJ7kK4U3Nwex9FKAa7lQJ593OHVjm6ZjqObPtswHZ9tYmLLdGyfGdCYFbaZrgeYrllMx/p+g4XsgKPzRE1zksA2p3BupTBHHwKgHo4GPRwy9nBoKeC9yfk26jQMddKw9Qkshjirr0WlGCvBbyc4bgA+f1tjzjs8KfmTmWc8Xh3PhZMqokE3WqxMi0A3FVwKKPBUTeGAFBCidl4OZ5Cr96cvvPfFKUOw/HCQ5fBDNM0+xmHaccgcXJ5Dx+EAqmTMSgPsWAomqBiZ9RDfZgOxrShg0XLED2BrIl0U96IAMwtHMp3SEa8hOZ5yfIZfZwBFUXfkJ57DxnooglVDrQAnwbzC4c4XAOZdWA5QghQlON+BLL4JR5LLb7jwr8RpNgtSb1sEdUJhnG64WWIVMaApoMBzQH/+Dr28mDyhJ2k5otQ97g9UZSnLXp6U6gsH0gwfS7ci0XRwoYMAdEPAEKZGOU8AKCaAY0zgNo5a7BqA5wMIwlEGhz2QA+7UMQGitdBPVmARQDW37VpwrDXM55QBOCuMsFU5PCVQnuuu4JFlt3BsML9qsGelRHH63xaeU1GKAoIC/x+temoEm1ZZSQAAAABJRU5ErkJggg==&quot; alt=&quot;Fig 1. Cassandra Streaming&quot; style=&quot;float: right;margin-right: 7px;margin-top: 7px;&quot; /&gt; To get an understanding of the impact of these changes, let’s first have a look at the current streaming code path. The diagram below illustrates the stream session setup when a node attempts to stream data from a peer. Let’s say, we have a 3 node cluster (Nodes A, B, C). Node C is being rebuilt and has to stream all data that it is responsible for from A &amp;amp; B. C setups a streaming session with each of it’s peers (See: &lt;a href=&quot;https://issues.apache.org/jira/browse/CASSANDRA-4650&quot;&gt;CASSANDRA-4560&lt;/a&gt; how Cassandra applies &lt;a href=&quot;https://en.wikipedia.org/wiki/Ford%E2%80%93Fulkerson_algorithm&quot;&gt;Ford Fulkerson&lt;/a&gt; to optimize streaming peers). It exchanges messages to request ranges and begins streaming data from the selected nodes.&lt;/p&gt;
&lt;p&gt;During the streaming phase, A collects all SSTables that have partitions in the requested ranges. It streams each SSTable by serializing individual partitions. Upon receiving the partition, node C reifies the data in memory and then writes it to disk. This is necessary to accurately transfer partitions from all possible SSTables for the requested ranges. This streaming path generates garbage and could be avoided in scenarios where all partitions within the SSTable need to be transmitted. This is common when you’re using LeveledCompactionStrategy or have enabled partitioning SSTables by token range (See: &lt;a href=&quot;http://issues.apache.org/jira/browse/CASSANDRA-6696&quot;&gt;CASSANDRA-6696&lt;/a&gt;), etc.&lt;/p&gt;
&lt;p&gt;To solve this problem &lt;a href=&quot;http://issues.apache.org/jira/browse/CASSANDRA-14556&quot;&gt;CASSANDRA-14556&lt;/a&gt; adds a Zero Copy streaming path. This significantly speeds up the transfer of SSTables and reduces garbage and unnecessary object creation. It modifies the streaming path to add additional information into the streaming header and uses ZeroCopy APIs to transfer bytes to and from the network and disk. So now, an SSTable may be transferred using this strategy when Cassandra detects that a complete SSTable needs to be transferred.&lt;/p&gt;
&lt;h2 id=&quot;how-do-i-use-this-feature&quot;&gt;How do I use this feature?&lt;/h2&gt;
&lt;p&gt;It just works. This feature is controlled using &lt;code class=&quot;highlighter-rouge&quot;&gt;stream_entire_sstables&lt;/code&gt; in &lt;code class=&quot;highlighter-rouge&quot;&gt;cassandra.yaml&lt;/code&gt; and is enabled by default. Even though this feature is enabled, it will respect the throttling limits as defined by &lt;code class=&quot;highlighter-rouge&quot;&gt;stream_throughput_outbound_megabits_per_sec&lt;/code&gt;.&lt;/p&gt;
&lt;h2 id=&quot;impact&quot;&gt;Impact&lt;/h2&gt;
&lt;p&gt;Cassandra can stream SSTables only bounded by the hardware limitations (Network and Disk IO). With this optimization, we hope to make Cassandra more performant and reliable.&lt;/p&gt;
&lt;p&gt;Microbenchmarking this feature shows a marked improvement (higher is better). Block Stream Writers are the ZeroCopy writers and Partial Stream Writers are the existing writers.&lt;/p&gt;
&lt;table class=&quot;table-condensed table-bordered table-hover&quot;&gt;
&lt;thead&gt;
&lt;tr&gt;
&lt;th&gt;Benchmark&lt;/th&gt;
&lt;th&gt;Mode&lt;/th&gt;
&lt;th&gt;Cnt&lt;/th&gt;
&lt;th&gt;Score&lt;/th&gt;
&lt;th&gt;Error&lt;/th&gt;
&lt;th&gt;Units&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr&gt;
&lt;td&gt;ZeroCopyStreamingBenchmark.blockStreamReader&lt;/td&gt;
&lt;td&gt;thrpt&lt;/td&gt;
&lt;td&gt;10&lt;/td&gt;
&lt;td&gt;20.119&lt;/td&gt;
&lt;td&gt;± 1.300&lt;/td&gt;
&lt;td&gt;ops/s&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;ZeroCopyStreamingBenchmark.blockStreamWriter&lt;/td&gt;
&lt;td&gt;thrpt&lt;/td&gt;
&lt;td&gt;10&lt;/td&gt;
&lt;td&gt;1339.672&lt;/td&gt;
&lt;td&gt;± 352.242&lt;/td&gt;
&lt;td&gt;ops/s&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;ZeroCopyStreamingBenchmark.partialStreamReader&lt;/td&gt;
&lt;td&gt;thrpt&lt;/td&gt;
&lt;td&gt;10&lt;/td&gt;
&lt;td&gt;0.590&lt;/td&gt;
&lt;td&gt;± 0.135&lt;/td&gt;
&lt;td&gt;ops/s&lt;/td&gt;
&lt;/tr&gt;
&lt;tr&gt;
&lt;td&gt;ZeroCopyStreamingBenchmark.partialStreamWriter&lt;/td&gt;
&lt;td&gt;thrpt&lt;/td&gt;
&lt;td&gt;10&lt;/td&gt;
&lt;td&gt;17.556&lt;/td&gt;
&lt;td&gt;± 0.323&lt;/td&gt;
&lt;td&gt;ops/s&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;h2 id=&quot;conclusion&quot;&gt;Conclusion&lt;/h2&gt;
&lt;p&gt;If you’re a Cassandra user, we would love to hear back from you. Please send us feedback via user &lt;a href=&quot;http://cassandra.apache.org/community/&quot;&gt;Mailing List&lt;/a&gt;, &lt;a href=&quot;https://issues.apache.org/jira/projects/CASSANDRA/summary&quot;&gt;Jira&lt;/a&gt;, or &lt;a href=&quot;http://cassandra.apache.org/community/&quot;&gt;IRC&lt;/a&gt; (or any combination of the three).&lt;/p&gt;</content><author><name>The Apache Cassandra Community</name></author><summary type="html">Streaming in Apache Cassandra powers host replacement, range movements, and cluster expansions. Streaming plays a crucial role in the cluster and as such its performance is key to not only the speed of the operations its used in but the cluster’s health generally. In Apache Cassandra 4.0, we have introduced an improved streaming implementation that reduces GC pressure and increases throughput several folds and are now limited, in some cases, only by the disk / network IO (See: CASSANDRA-14556).</summary></entry></feed>