From 8f806ab48643b16a975691bf6edf7887706327f1 Mon Sep 17 00:00:00 2001 From: Frank Welsch Date: Fri, 23 Sep 2016 18:00:42 -0400 Subject: [PATCH] HBASE-16574 Book updates for backup and restore Signed-off-by: Josh Elser --- .../asciidoc/_chapters/backup_restore.adoc | 912 ++++++++++++++++++ src/main/asciidoc/book.adoc | 5 +- .../images/backup-app-components.png | Bin 0 -> 24366 bytes .../images/backup-cloud-appliance.png | Bin 0 -> 30114 bytes .../images/backup-dedicated-cluster.png | Bin 0 -> 24950 bytes .../resources/images/backup-intra-cluster.png | Bin 0 -> 19348 bytes 6 files changed, 914 insertions(+), 3 deletions(-) create mode 100644 src/main/asciidoc/_chapters/backup_restore.adoc create mode 100644 src/main/site/resources/images/backup-app-components.png create mode 100644 src/main/site/resources/images/backup-cloud-appliance.png create mode 100644 src/main/site/resources/images/backup-dedicated-cluster.png create mode 100644 src/main/site/resources/images/backup-intra-cluster.png diff --git a/src/main/asciidoc/_chapters/backup_restore.adoc b/src/main/asciidoc/_chapters/backup_restore.adoc new file mode 100644 index 00000000000..a9dbcf5ec12 --- /dev/null +++ b/src/main/asciidoc/_chapters/backup_restore.adoc @@ -0,0 +1,912 @@ +//// +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +//// + +[[casestudies]] += Backup and Restore +:doctype: book +:numbered: +:toc: left +:icons: font +:experimental: + +[[br.overview]] +== Overview + +Backup and restore is a standard operation provided by many databases. An effective backup and restore +strategy helps ensure that users can recover data in case of unexpected failures. The HBase backup and restore +feature helps ensure that enterprises using HBase as a canonical data repository can recover from catastrophic +failures. Another important feature is the ability to restore the database to a particular +point-in-time, commonly referred to as a snapshot. + +The HBase backup and restore feature provides the ability to create full backups and incremental backups on +tables in an HBase cluster. The full backup is the foundation on which incremental backups are applied +to build iterative snapshots. Incremental backups can be run on a schedule to capture changes over time, +for example by using a Cron task. Incremental backups are more cost-effective than full backups because they only capture +the changes since the last backup and they also enable administrators to restore the database to any prior incremental backup. Furthermore, the +utilities also enable table-level data backup-and-recovery if you do not want to restore the entire dataset +of the backup. + +The backup and restore feature supplements the HBase Replication feature. While HBase replication is ideal for +creating "hot" copies of the data (where the replicated data is immediately available for query), the backup and +restore feature is ideal for creating "cold" copies of data (where a manual step must be taken to restore the system). +Previously, users only had the ability to create full backups via the ExportSnapshot functionality. The incremental +backup implementation is the novel improvement over the previous "art" provided by ExportSnapshot. + +[[br.terminology]] +== Terminology + +The backup and restore feature introduces new terminology which can be used to understand how control flows through the +system. + +* _A backup_: A logical unit of data and metadata which can restore a table to its state at a specific point in time. +* _Full backup_: a type of backup which wholly encapsulates the contents of the table at a point in time. +* _Incremental backup_: a type of backup which contains the changes in a table since a full backup. +* _Backup set_: A user-defined name which references one or more tables over which a backup can be executed. +* _Backup ID_: A unique names which identifies one backup from the rest, e.g. `backupId_1467823988425` + +[[br.planning]] +== Planning + +There are some common strategies which can be used to implement backup and restore in your environment. The following section +shows how these strategies are implemented and identifies potential tradeoffs with each. + +WARNING: This backup and restore tools has not been tested on Transparent Data Encryption (TDE) enabled HDFS clusters. +This is related to the open issue link:https://issues.apache.org/jira/browse/HBASE-16178[HBASE-16178]. + +[[br.intracluster.backup]] +=== Backup within a cluster + +This strategy stores the backups on the same cluster as where the backup was taken. This approach is only appropriate for testing +as it does not provide any additional safety on top of what the software itself already provides. + +.Intra-Cluster Backup +image::backup-intra-cluster.png[] + +[[br.dedicated.cluster.backup]] +=== Backup using a dedicated cluster + +This strategy provides greater fault tolerance and provides a path towards disaster recovery. In this setting, you will +store the backup on a separate HDFS cluster by supplying the backup destination cluster’s HDFS URL to the backup utility. +You should consider backing up to a different physical location, such as a different data center. + +Typically, a backup-dedicated HDFS cluster uses a more economical hardware profile to save money. + +.Dedicated HDFS Cluster Backup +image::backup-dedicated-cluster.png[] + +[[br.cloud.or.vendor.backup]] +=== Backup to the Cloud or a storage vendor appliance + +Another approach to safeguarding HBase incremental backups is to store the data on provisioned, secure servers that belong +to third-party vendors and that are located off-site. The vendor can be a public cloud provider or a storage vendor who uses +a Hadoop-compatible file system, such as S3 and other HDFS-compatible destinations. + +.Backup to Cloud or Vendor Storage Solutions +image::backup-cloud-appliance.png[] + +NOTE: The HBase backup utility does not support backup to multiple destinations. A workaround is to manually create copies +of the backup files from HDFS or S3. + +[[br.initial.setup]] +== First-time configuration steps + +This section contains the necessary configuration changes that must be made in order to use the backup and restore feature. +As this feature makes significant use of YARN's MapReduce framework to parallelize these I/O heavy operations, configuration +changes extend outside of just `hbase-site.xml`. + +=== Allow the "hbase" system user in YARN + +The YARN *container-executor.cfg* configuration file must have the following property setting: _allowed.system.users=hbase_. No spaces +are allowed in entries of this configuration file. + +WARNING: Skipping this step will result in runtime errors when executing the first backup tasks. + +*Example of a valid container-executor.cfg file for backup and restore:* + +[source] +---- +yarn.nodemanager.log-dirs=/var/log/hadoop/mapred +yarn.nodemanager.linux-container-executor.group=yarn +banned.users=hdfs,yarn,mapred,bin +allowed.system.users=hbase +min.user.id=500 +---- + +=== HBase specific changes + +Add the following properties to hbase-site.xml and restart HBase if it is already running. + +NOTE: The ",..." is an ellipsis meant to imply that this is a comma-separated list of values, not literal text which should be added to hbase-site.xml. + +[source] +---- + + hbase.backup.enable + true + + + hbase.master.logcleaner.plugins + org.apache.hadoop.hbase.backup.master.BackupLogCleaner,... + + + hbase.procedure.master.classes + org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager,... + + + hbase.procedure.regionserver.classes + org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager,... + + + hbase.coprocessor.region.classes + org.apache.hadoop.hbase.backup.BackupObserver,... + + + hbase.master.hfilecleaner.plugins + org.apache.hadoop.hbase.backup.BackupHFileCleaner,... + +---- + +== Backup and Restore commands + +This covers the command-line utilities that administrators would run to create, restore, and merge backups. Tools to +inspect details on specific backup sessions is covered in the next section, <>. + +Run the command `hbase backup help ` to access the online help that provides basic information about a command +and its options. The below information is captured in this help message for each command. + +// hbase backup create + +[[br.creating.complete.backup]] +### Creating a Backup Image + +[NOTE] +==== +For HBase clusters also using Apache Phoenix: include the SQL system catalog tables in the backup. In the event that you +need to restore the HBase backup, access to the system catalog tables enable you to resume Phoenix interoperability with the +restored data. +==== + +The first step in running the backup and restore utilities is to perform a full backup and to store the data in a separate image +from the source. At a minimum, you must do this to get a baseline before you can rely on incremental backups. + +Run the following command as HBase superuser: + +[source] +---- +hbase backup create +---- + +After the command finishes running, the console prints a SUCCESS or FAILURE status message. The SUCCESS message includes a _backup_ ID. +The backup ID is the Unix time (also known as Epoch time) that the HBase master received the backup request from the client. + +[TIP] +==== +Record the backup ID that appears at the end of a successful backup. In case the source cluster fails and you need to recover the +dataset with a restore operation, having the backup ID readily available can save time. +==== + +[[br.create.positional.cli.arguments]] +#### Positional Command-Line Arguments + +_type_:: + The type of backup to execute: _full_ or _incremental_. As a reminder, an _incremental_ backup requires a _full_ backup to + already exist. + +_backup_path_:: + The _backup_path_ argument specifies the full filesystem URI of where to store the backup image. Valid prefixes are + are _hdfs:_, _webhdfs:_, _gpfs:_, and _s3fs:_. + +[[br.create.named.cli.arguments]] +#### Named Command-Line Arguments + +_-t _:: + A comma-separated list of tables to back up. If no tables are specified, all tables are backed up. No regular-expression or + wildcard support is present; all table names must be explicitly listed. See <> for more + information about peforming operations on collections of tables. Mutually exclusive with the _-s_ option; one of these + named options are required. + +_-s _:: + Identify tables to backup based on a backup set. See <> for the purpose and usage + of backup sets. Mutually exclusive with the _-t_ option. + +_-w _:: + (Optional) Specifies the number of parallel workers to copy data to backup destination. Backups are currently executed by MapReduce jobs + so this value corresponds to the number of Mappers that will be spawned by the job. + +_-b _:: + (Optional) Specifies the bandwidth of each worker in MB per second. + +_-d_:: + (Optional) Enables "DEBUG" mode which prints additional logging about the backup creation. + +_-q _:: + (Optional) Allows specification of the name of a YARN queue which the MapReduce job to create the backup should be executed in. This option + is useful to prevent backup tasks from stealing resources away from other MapReduce jobs of high importance. + +[[br.usage.examples]] +#### Example usage + +[source] +---- +$ hbase backup create full hdfs://host5:8020/data/backup -t SALES2,SALES3 -w 3 +---- + +This command creates a full backup image of two tables, SALES2 and SALES3, in the HDFS instance who NameNode is host5:8020 +in the path _/data/backup_. The _-w_ option specifies that no more than three parallel works complete the operation. + +// hbase backup restore + +[[br.restoring.backup]] +### Restoring a Backup Image + +Run the following command as an HBase superuser. You can only restore a backup on a running HBase cluster because the data must be +redistributed the RegionServers for the operation to complete successfully. + +[source] +---- +hbase restore +---- + +[[br.restore.positional.args]] +#### Positional Command-Line Arguments + +_backup_path_:: + The _backup_path_ argument specifies the full filesystem URI of where to store the backup image. Valid prefixes are + are _hdfs:_, _webhdfs:_, _gpfs:_, and _s3fs:_. + +_backup_id_:: + The backup ID that uniquely identifies the backup image to be restored. + + +[[br.restore.named.args]] +#### Named Command-Line Arguments + +_-t _:: + A comma-separated list of tables to restore. See <> for more + information about peforming operations on collections of tables. Mutually exclusive with the _-s_ option; one of these + named options are required. + +_-s _:: + Identify tables to backup based on a backup set. See <> for the purpose and usage + of backup sets. Mutually exclusive with the _-t_ option. + +_-q _:: + (Optional) Allows specification of the name of a YARN queue which the MapReduce job to create the backup should be executed in. This option + is useful to prevent backup tasks from stealing resources away from other MapReduce jobs of high importance. + +_-c_:: + (Optional) Perform a dry-run of the restore. The actions are checked, but not executed. + +_-m _:: + (Optional) A comma-separated list of tables to restore into. If this option is not provided, the original table name is used. When + this option is provided, there must be an equal number of entries provided in the `-t` option. + +_-o_:: + (Optional) Overwrites the target table for the restore if the table already exists. + + +[[br.restore.usage]] +#### Example of Usage + +[source] +---- +hbase backup restore /tmp/backup_incremental backupId_1467823988425 -t mytable1,mytable2 +---- + +This command restores two tables of an incremental backup image. In this example: +• `/tmp/backup_incremental` is the path to the directory containing the backup image. +• `backupId_1467823988425` is the backup ID. +• `mytable1` and `mytable2` are the names of tables in the backup image to be restored. + +// hbase backup merge + +[[br.merge.backup]] +### Merging Incremental Backup Images + +This command can be used to merge two or more incremental backup images into a single incremental +backup image. This can be used to consolidate multiple, small incremental backup images into a single +larger incremental backup image. This command could be used to merge hourly incremental backups +into a daily incremental backup image, or daily incremental backups into a weekly incremental backup. + +[source] +---- +$ hbase backup merge +---- + +[[br.merge.backup.positional.cli.arguments]] +#### Positional Command-Line Arguments + +_backup_ids_:: + A comma-separated list of incremental backup image IDs that are to be combined into a single image. + +[[br.merge.backup.named.cli.arguments]] +#### Named Command-Line Arguments + +None. + +[[br.merge.backup.example]] +#### Example usage + +[source] +---- +$ hbase backup merge backupId_1467823988425,backupId_1467827588425 +---- + +// hbase backup set + +[[br.using.backup.sets]] +### Using Backup Sets + +Backup sets can ease the administration of HBase data backups and restores by reducing the amount of repetitive input +of table names. You can group tables into a named backup set with the `hbase backup set add` command. You can then use +the -set option to invoke the name of a backup set in the `hbase backup create` or `hbase backup restore` rather than list +individually every table in the group. You can have multiple backup sets. + +NOTE: Note the differentiation between the `hbase backup set add` command and the _-set_ option. The `hbase backup set add` +command must be run before using the `-set` option in a different command because backup sets must be named and defined +before using backup sets as a shortcut. + +If you run the `hbase backup set add` command and specify a backup set name that does not yet exist on your system, a new set +is created. If you run the command with the name of an existing backup set name, then the tables that you specify are added +to the set. + +In this command, the backup set name is case-sensitive. + +NOTE: The metadata of backup sets are stored within HBase. If you do not have access to the original HBase cluster with the +backup set metadata, then you must specify individual table names to restore the data. + +To create a backup set, run the following command as the HBase superuser: + +[source] +---- +$ hbase backup set +---- + +[[br.set.subcommands]] +#### Backup Set Subcommands + +The following list details subcommands of the hbase backup set command. + +NOTE: You must enter one (and no more than one) of the following subcommands after hbase backup set to complete an operation. +Also, the backup set name is case-sensitive in the command-line utility. + +_add_:: + Adds table[s] to a backup set. Specify a _backup_set_name_ value after this argument to create a backup set. + +_remove_:: + Removes tables from the set. Specify the tables to remove in the tables argument. + +_list_:: + Lists all backup sets. + +_describe_:: + Displays a description of a backup set. The information includes whether the set has full + or incremental backups, start and end times of the backups, and a list of the tables in the set. This subcommand must precede + a valid value for the _backup_set_name_ value. + +_delete_:: + Deletes a backup set. Enter the value for the _backup_set_name_ option directly after the `hbase backup set delete` command. + +[[br.set.positional.cli.arguments]] +#### Positional Command-Line Arguments + +_backup_set_name_:: + Use to assign or invoke a backup set name. The backup set name must contain only printable characters and cannot have any spaces. + +_tables_:: + List of tables (or a single table) to include in the backup set. Enter the table names as a comma-separated list. If no tables + are specified, all tables are included in the set. + +TIP: Maintain a log or other record of the case-sensitive backup set names and the corresponding tables in each set on a separate +or remote cluster, backup strategy. This information can help you in case of failure on the primary cluster. + +[[br.set.usage]] +#### Example of Usage + +[source] +---- +$ hbase backup set add Q1Data TEAM3,TEAM_4 +---- + +Depending on the environment, this command results in _one_ of the following actions: + +* If the `Q1Data` backup set does not exist, a backup set containing tables `TEAM_3` and `TEAM_4` is created. +* If the `Q1Data` backup set exists already, the tables `TEAM_3` and `TEAM_4` are added to the `Q1Data` backup set. + +[[br.administration]] +## Administration of Backup Images + +The `hbase backup` command has several subcommands that help with administering backup images as they accumulate. Most production +environments require recurring backups, so it is necessary to have utilities to help manage the data of the backup repository. +Some subcommands enable you to find information that can help identify backups that are relevant in a search for particular data. +You can also delete backup images. + +The following list details each `hbase backup subcommand` that can help administer backups. Run the full command-subcommand line as +the HBase superuser. + +// hbase backup progress + +[[br.managing.backup.progress]] +### Managing Backup Progress + +You can monitor a running backup in another terminal session by running the _hbase backup progress_ command and specifying the backup ID as an argument. + +For example, run the following command as hbase superuser to view the progress of a backup + +[source] +---- +$ hbase backup progress +---- + +[[br.progress.positional.cli.arguments]] +#### Positional Command-Line Arguments + +_backup_id_:: + Specifies the backup that you want to monitor by seeing the progress information. The backupId is case-sensitive. + +[[br.progress.named.cli.arguments]] +#### Named Command-Line Arguments + +None. + +[[br.progress.example]] +#### Example usage + +[source] +---- +hbase backup progress backupId_1467823988425 +---- + +// hbase backup history + +[[br.managing.backup.history]] +### Managing Backup History + +This command displays a log of backup sessions. The information for each session includes backup ID, type (full or incremental), the tables +in the backup, status, and start and end time. Specify the number of backup sessions to display with the optional -n argument. + +[source] +---- +$ hbase backup history +---- + +[[br.history.positional.cli.arguments]] +#### Positional Command-Line Arguments + +_backup_id_:: + Specifies the backup that you want to monitor by seeing the progress information. The backupId is case-sensitive. + +[[br.history.named.cli.arguments]] +#### Named Command-Line Arguments + +_-n _:: + (Optional) The maximum number of backup records (Default: 10). + +_-p _:: + The full filesystem URI of where backup images are stored. + +_-s _:: + The name of the backup set to obtain history for. Mutually exclusive with the _-t_ option. + +_-t_ :: + The name of table to obtain history for. Mutually exclusive with the _-s_ option. + +[[br.history.backup.example]] +#### Example usage + +[source] +---- +$ hbase backup history +$ hbase backup history -n 20 +$ hbase backup history -t WebIndexRecords +---- + +// hbase backup describe + +[[br.describe.backup]] +### Describing a Backup Image + +This command can be used to obtain information about a specific backup image. + +[source] +---- +$ hbase backup describe +---- + +[[br.describe.backup.positional.cli.arguments]] +#### Positional Command-Line Arguments + +_backup_id_:: + The ID of the backup image to describe. + +[[br.describe.backup.named.cli.arguments]] +#### Named Command-Line Arguments + +None. + +[[br.describe.backup.example]] +#### Example usage + +[source] +---- +$ hbase backup describe backupId_1467823988425 +---- + +// hbase backup delete + +[[br.delete.backup]] +### Deleting a Backup Image + +This command can be used to delete a backup image which is no longer needed. + +[source] +---- +$ hbase backup delete +---- + +[[br.delete.backup.positional.cli.arguments]] +#### Positional Command-Line Arguments + +_backup_id_:: + The ID to the backup image which should be deleted. + +[[br.delete.backup.named.cli.arguments]] +#### Named Command-Line Arguments + +None. + +[[br.delete.backup.example]] +#### Example usage + +[source] +---- +$ hbase backup delete backupId_1467823988425 +---- + +// hbase backup repair + +[[br.repair.backup]] +### Backup Repair Command + +This command attempts to correct any inconsistencies in persisted backup metadata which exists as +the result of software errors or unhandled failure scenarios. While the backup implementation tries +to correct all errors on its own, this tool may be necessary in the cases where the system cannot +automatically recover on its own. + +[source] +---- +$ hbase backup repair +---- + +[[br.repair.backup.positional.cli.arguments]] +#### Positional Command-Line Arguments + +None. + +[[br.repair.backup.named.cli.arguments]] +### Named Command-Line Arguments + +None. + +[[br.repair.backup.example]] +#### Example usage + +[source] +---- +$ hbase backup repair +---- + +[[br.backup.configuration]] +## Configuration keys + +The backup and restore feature includes both required and optional configuration keys. + +### Required properties + +_hbase.backup.enable_: Controls whether or not the feature is enabled (Default: `false`). Set this value to `true`. + +_hbase.master.logcleaner.plugins_: A comma-separated list of classes invoked when cleaning logs in the HBase Master. Set +this value to `org.apache.hadoop.hbase.backup.master.BackupLogCleaner` or append it to the current value. + +_hbase.procedure.master.classes_: A comma-separated list of classes invoked with the Procedure framework in the Master. Set +this value to `org.apache.hadoop.hbase.backup.master.LogRollMasterProcedureManager` or append it to the current value. + +_hbase.procedure.regionserver.classes_: A comma-separated list of classes invoked with the Procedure framework in the RegionServer. +Set this value to `org.apache.hadoop.hbase.backup.regionserver.LogRollRegionServerProcedureManager` or append it to the current value. + +_hbase.coprocessor.region.classes_: A comma-separated list of RegionObservers deployed on tables. Set this value to +`org.apache.hadoop.hbase.backup.BackupObserver` or append it to the current value. + +_hbase.master.hfilecleaner.plugins_: A comma-separated list of HFileCleaners deployed on the Master. Set this value +to `org.apache.hadoop.hbase.backup.BackupHFileCleaner` or append it to the current value. + +### Optional properties + +_hbase.backup.system.ttl_: The time-to-live in seconds of data in the `hbase:backup` tables (default: forever). This property +is only relevant prior to the creation of the `hbase:backup` table. Use the `alter` command in the HBase shell to modify the TTL +when this table already exists. See the <> for more details on the impact of this +configuration property. + +_hbase.backup.attempts.max_: The number of attempts to perform when taking hbase table snapshots (default: 10). + +_hbase.backup.attempts.pause.ms_: The amount of time to wait between failed snapshot attempts in milliseconds (default: 10000). + +_hbase.backup.logroll.timeout.millis_: The amount of time (in milliseconds) to wait for RegionServers to execute a WAL rolling +in the Master's procedure framework (default: 30000). + +[[br.best.practices]] +## Best Practices + +### Formulate a restore strategy and test it. + +Before you rely on a backup and restore strategy for your production environment, identify how backups must be performed, +and more importantly, how restores must be performed. Test the plan to ensure that it is workable. +At a minimum, store backup data from a production cluster on a different cluster or server. To further safeguard the data, +use a backup location that is at a different physical location. + +If you have a unrecoverable loss of data on your primary production cluster as a result of computer system issues, you may +be able to restore the data from a different cluster or server at the same site. However, a disaster that destroys the whole +site renders locally stored backups useless. Consider storing the backup data and necessary resources (both computing capacity +and operator expertise) to restore the data at a site sufficiently remote from the production site. In the case of a catastrophe +at the whole primary site (fire, earthquake, etc.), the remote backup site can be very valuable. + +### Secure a full backup image first. + +As a baseline, you must complete a full backup of HBase data at least once before you can rely on incremental backups. The full +backup should be stored outside of the source cluster. To ensure complete dataset recovery, you must run the restore utility +with the option to restore baseline full backup. The full backup is the foundation of your dataset. Incremental backup data +is applied on top of the full backup during the restore operation to return you to the point in time when backup was last taken. + +### Define and use backup sets for groups of tables that are logical subsets of the entire dataset. + +You can group tables into an object called a backup set. A backup set can save time when you have a particular group of tables +that you expect to repeatedly back up or restore. + +When you create a backup set, you type table names to include in the group. The backup set includes not only groups of related +tables, but also retains the HBase backup metadata. Afterwards, you can invoke the backup set name to indicate what tables apply +to the command execution instead of entering all the table names individually. + +### Document the backup and restore strategy, and ideally log information about each backup. + +Document the whole process so that the knowledge base can transfer to new administrators after employee turnover. As an extra +safety precaution, also log the calendar date, time, and other relevant details about the data of each backup. This metadata +can potentially help locate a particular dataset in case of source cluster failure or primary site disaster. Maintain duplicate +copies of all documentation: one copy at the production cluster site and another at the backup location or wherever it can be +accessed by an administrator remotely from the production cluster. + +[[br.s3.backup.scenario]] +## Scenario: Safeguarding Application Datasets on Amazon S3 + +This scenario describes how a hypothetical retail business uses backups to safeguard application data and then restore the dataset +after failure. + +The HBase administration team uses backup sets to store data from a group of tables that have interrelated information for an +application called green. In this example, one table contains transaction records and the other contains customer details. The +two tables need to be backed up and be recoverable as a group. + +The admin team also wants to ensure daily backups occur automatically. + +.Tables Composing The Backup Set +image::backup-app-components.png[] + +The following is an outline of the steps and examples of commands that are used to backup the data for the _green_ application and +to recover the data later. All commands are run when logged in as HBase superuser. + +1. A backup set called _green_set_ is created as an alias for both the transactions table and the customer table. The backup set can +be used for all operations to avoid typing each table name. The backup set name is case-sensitive and should be formed with only +printable characters and without spaces. + +[source] +---- +$ hbase backup set add green_set transactions +$ hbase backup set add green_set customer +---- + +2. The first backup of green_set data must be a full backup. The following command example shows how credentials are passed to Amazon +S3 and specifies the file system with the s3a: prefix. + +[source] +---- +$ ACCESS_KEY=ABCDEFGHIJKLMNOPQRST +$ SECRET_KEY=123456789abcdefghijklmnopqrstuvwxyzABCD +$ sudo -u hbase hbase backup create full\ + s3a://$ACCESS_KEY:SECRET_KEY@prodhbasebackups/backups -s green_set +---- + +3. Incremental backups should be run according to a schedule that ensures essential data recovery in the event of a catastrophe. At +this retail company, the HBase admin team decides that automated daily backups secures the data sufficiently. The team decides that +they can implement this by modifying an existing Cron job that is defined in `/etc/crontab`. Consequently, IT modifies the Cron job +by adding the following line: + +[source] +---- +@daily hbase hbase backup create incremental s3a://$ACCESS_KEY:$SECRET_KEY@prodhbasebackups/backups -s green_set +---- + +4. A catastrophic IT incident disables the production cluster that the green application uses. An HBase system administrator of the +backup cluster must restore the _green_set_ dataset to the point in time closest to the recovery objective. + +NOTE: If the administrator of the backup HBase cluster has the backup ID with relevant details in accessible records, the following +search with the `hdfs dfs -ls` command and manually scanning the backup ID list can be bypassed. Consider continuously maintaining +and protecting a detailed log of backup IDs outside the production cluster in your environment. + +The HBase administrator runs the following command on the directory where backups are stored to print the list of successful backup +IDs on the console: + +`hdfs dfs -ls -t /prodhbasebackups/backups` + +5. The admin scans the list to see which backup was created at a date and time closest to the recovery objective. To do this, the +admin converts the calendar timestamp of the recovery point in time to Unix time because backup IDs are uniquely identified with +Unix time. The backup IDs are listed in reverse chronological order, meaning the most recent successful backup appears first. + +The admin notices that the following line in the command output corresponds with the _green_set_ backup that needs to be restored: + +`/prodhbasebackups/backups/backup_1467823988425` + +6. The admin restores green_set invoking the backup ID and the -overwrite option. The -overwrite option truncates all existing data +in the destination and populates the tables with data from the backup dataset. Without this flag, the backup data is appended to the +existing data in the destination. In this case, the admin decides to overwrite the data because it is corrupted. + +[source] +---- +$ sudo -u hbase hbase restore -s green_set \ + s3a://$ACCESS_KEY:$SECRET_KEY@prodhbasebackups/backups backup_1467823988425 \ -overwrite +---- + +[[br.data.security]] +## Security of Backup Data + +With this feature which makes copying data to remote locations, it's important to take a moment to clearly state the procedural +concerns that exist around data security. Like the HBase replication feature, backup and restore provides the constructs to automatically +copy data from within a corporate boundary to some system outside of that boundary. It is imperative when storing sensitive data that with backup and restore, much +less any feature which extracts data from HBase, the locations to which data is being sent has undergone a security audit to ensure +that only authenticated users are allowed to access that data. + +For example, with the above example of backing up data to S3, it is of the utmost importance that the proper permissions are assigned +to the S3 bucket to ensure that only a minimum set of authorized users are allowed to access this data. Because the data is no longer +being accessed via HBase, and its authentication and authorization controls, we must ensure that the filesystem storing that data is +providing a comparable level of security. This is a manual step which users *must* implement on their own. + +[[br.technical.details]] +## Technical Details of Incremental Backup and Restore + +HBase incremental backups enable more efficient capture of HBase table images than previous attempts at serial backup and restore +solutions, such as those that only used HBase Export and Import APIs. Incremental backups use Write Ahead Logs (WALs) to capture +the data changes since the previous backup was created. A WAL roll (create new WALs) is executed across all RegionServers to track +the WALs that need to be in the backup. + +After the incremental backup image is created, the source backup files usually are on same node as the data source. A process similar +to the DistCp (distributed copy) tool is used to move the source backup files to the target file systems. When a table restore operation +starts, a two-step process is initiated. First, the full backup is restored from the full backup image. Second, all WAL files from +incremental backups between the last full backup and the incremental backup being restored are converted to HFiles, which the HBase +Bulk Load utility automatically imports as restored data in the table. + +You can only restore on a live HBase cluster because the data must be redistributed to complete the restore operation successfully. + +[[br.filesystem.growth.warning]] +## A Warning on File System Growth + +As a reminder, incremental backups are implemented via retaining the write-ahead logs which HBase primarily uses for data durability. +Thus, to ensure that all data needing to be included in a backup is still available in the system, the HBase backup and restore feature +retains all write-ahead logs since the last backup until the next incremental backup is executed. + +Like HBase Snapshots, this can have an expectedly large impact on the HDFS usage of HBase for high volume tables. Take care in enabling +and using the backup and restore feature, specifically with a mind to removing backup sessions when they are not actively being used. + +The only automated, upper-bound on retained write-ahead logs for backup and restore is based on the TTL of the `hbase:backup` system table which, +as of the time this document is written, is infinite (backup table entries are never automatically deleted). This requires that administrators +perform backups on a schedule whose frequency is relative to the amount of available space on HDFS (e.g. less available HDFS space requires +more aggressive backup merges and deletions). As a reminder, the TTL can be altered on the `hbase:backup` table using the `alter` command +in the HBase shell. Modifying the configuration property `hbase.backup.system.ttl` in hbase-site.xml after the system table exists has no effect. + +[[br.backup.capacity.planning]] +## Capacity Planning + +When designing a distributed system deployment, it is critical that some basic mathmatical rigor is executed to ensure sufficient computational +capacity is available given the data and software requirements of the system. For this feature, the availability of network capacity is the largest +bottleneck when estimating the performance of some implementation of backup and restore. The second most costly function is the speed at which +data can be read/written. + +### Full Backups + +To estimate the duration of a full backup, we have to understand the general actions which are invoked: + +* Write-ahead log roll on each RegionServer: ones to tens of seconds per RegionServer in parallel. Relative to the load on each RegionServer. +* Take an HBase snapshot of the table(s): tens of seconds. Relative to the number of regions and files that comprise the table. +* Export the snapshot to the destination: see below. Relative to the size of the data and the network bandwidth to the destination. + +[[br.export.snapshot.cost]] +To approximate how long the final step will take, we have to make some assumptions on hardware. Be aware that these will *not* be accurate for your +system -- these are numbers that your or your administrator know for your system. Let's say the speed of reading data from HDFS on a single node is +capped at 80MB/s (across all Mappers that run on that host), a modern network interface controller (NIC) supports 10Gb/s, the top-of-rack switch can +handle 40Gb/s, and the WAN between your clusters is 10Gb/s. This means that you can only ship data to your remote at a speed of 1.25GB/s -- meaning +that 16 nodes (`1.25 * 1024 / 80 = 16`) participating in the ExportSnapshot should be able to fully saturate the link between clusters. With more +nodes in the cluster, we can still saturate the network but at a lesser impact on any one node which helps ensure local SLAs are made. If the size +of the snapshot is 10TB, this would full backup would take in the ballpark of 2.5 hours (`10 * 1024 / 1.25 / (60 * 60) = 2.23hrs`) + +As a general statement, it is very likely that the WAN bandwidth between your local cluster and the remote storage is the largest +bottleneck to the speed of a full backup. + +When the concern is restricting the computational impact of backups to a "production system", the above formulas can be reused with the optional +command-line arguments to `hbase backup create`: `-b`, `-w`, `-q`. The `-b` option defines the bandwidth at which each worker (Mapper) would +write data. The `-w` argument limits the number of workers that would be spawned in the DistCp job. The `-q` allows the user to specify a YARN +queue which can limit the specific nodes where the workers will be spawned -- this can quarantine the backup workers performing the copy to +a set of non-critical nodes. Relating the `-b` and `-w` options to our earlier equations: `-b` would be used to restrict each node from reading +data at the full 80MB/s and `-w` is used to limit the job from spawning 16 worker tasks. + +### Incremental Backup + +Like we did for full backups, we have to understand the incremental backup process to approximate its runtime and cost. + +* Identify new write-ahead logs since last full or incremental backup: negligible. Apriori knowledge from the backup system table(s). +* Read, filter, and write "minimized" HFiles equivalent to the WALs: dominated by the speed of writing data. Relative to write speed of HDFS. +* DistCp the HFiles to the destination: <>. + +For the second step, the dominating cost of this operation would be the re-writing the data (under the assumption that a majority of the +data in the WAL is preserved). In this case, we can assume an aggregate write speed of 30MB/s per node. Continuing our 16-node cluster example, +this would require approximately 15 minutes to perform this step for 50GB of data (50 * 1024 / 60 / 60 = 14.2). The amount of time to start the +DistCp MapReduce job would likely dominate the actual time taken to copy the data (50 / 1.25 = 40 seconds) and can be ignored. + +[[br.limitations]] +## Limitations of the Backup and Restore Utility + +*Serial backup operations* + +Backup operations cannot be run concurrently. An operation includes actions like create, delete, restore, and merge. Only one active backup session is supported. link:https://issues.apache.org/jira/browse/HBASE-16391[HBASE-16391] +will introduce multiple-backup sessions support. + +*No means to cancel backups* + +Both backup and restore operations cannot be canceled. (link:https://issues.apache.org/jira/browse/HBASE-15997[HBASE-15997], link:https://issues.apache.org/jira/browse/HBASE-15998[HBASE-15998]). +The workaround to cancel a backup would be to kill the client-side backup command (`control-C`), ensure all relevant MapReduce jobs have exited, and then +run the `hbase backup repair` command to ensure the system backup metadata is consistent. + +*Backups can only be saved to a single location* + +Copying backup information to multiple locations is an exercise left to the user. link:https://issues.apache.org/jira/browse/HBASE-15476[HBASE-15476] will +introduce the ability to specify multiple-backup destinations intrinsically. + +*HBase superuser access is required* + +Only an HBase superuser (e.g. hbase) is allowed to perform backup/restore, can pose a problem for shared HBase installations. Current mitigations would require +coordination with system administrators to build and deploy a backup and restore strategy (link:https://issues.apache.org/jira/browse/HBASE-14138[HBASE-14138]). + +*Backup restoration is an online operation* + +To perform a restore from a backup, it requires that the HBase cluster is online as a caveat of the current implementation (link:https://issues.apache.org/jira/browse/HBASE-16573[HBASE-16573]). + +*Some operations may fail and require re-run* + +The HBase backup feature is primarily client driven. While there is the standard HBase retry logic built into the HBase Connection, persistent errors in executing operations +may propagate back to the client (e.g. snapshot failure due to region splits). The backup implementation should be moved from client-side into the ProcedureV2 framework +in the future which would provide additional robustness around transient/retryable failures. The `hbase backup repair` command is meant to correct states which the system +cannot automatically detect and recover from. + +*Avoidance of declaration of public API* + +While the Java API to interact with this feature exists and its implementation is separated from an interface, insufficient rigor has been applied to determine if +it is exactly what we intend to ship to users. As such, it is marked as for a `Private` audience with the expectation that, as users begin to try the feature, there +will be modifications that would necessitate breaking compatibility (link:https://issues.apache.org/jira/browse/HBASE-17517[HBASE-17517]). + +*Lack of global metrics for backup and restore* + +Individual backup and restore operations contain metrics about the amount of work the operation included, but there is no centralized location (e.g. the Master UI) +which present information for consumption (link:https://issues.apache.org/jira/browse/HBASE-16565[HBASE-16565]). diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc index 519cf9afa31..1bc9ed7d88e 100644 --- a/src/main/asciidoc/book.adoc +++ b/src/main/asciidoc/book.adoc @@ -19,7 +19,7 @@ */ //// -= Apache HBase (TM) Reference Guide += Apache HBase (TM) Reference Guide :Author: Apache HBase Team :Email: :doctype: book @@ -62,6 +62,7 @@ include::_chapters/mapreduce.adoc[] include::_chapters/security.adoc[] include::_chapters/architecture.adoc[] include::_chapters/hbase_mob.adoc[] +include::_chapters/backup_restore.adoc[] include::_chapters/hbase_apis.adoc[] include::_chapters/external_apis.adoc[] include::_chapters/thrift_filter_language.adoc[] @@ -93,5 +94,3 @@ include::_chapters/asf.adoc[] include::_chapters/orca.adoc[] include::_chapters/tracing.adoc[] include::_chapters/rpc.adoc[] - - diff --git a/src/main/site/resources/images/backup-app-components.png b/src/main/site/resources/images/backup-app-components.png new file mode 100644 index 0000000000000000000000000000000000000000..5e403e25c6c917ffb34c68a98aae974cb873f69b GIT binary patch literal 24366 zcmZ6z18`(r)Gi#`wr$(Ct%;q9lZkEHww;M>+nCsx*ttFLz5iEV{aw}7)#p^7v$4)z z>%r~_B?U=%7#tWNARu^YDKQlwAYcl>8ygA&aD?YHyb$mK?5rXw0#rMLcLoF`1SBmc zto94|DhDzO*1z>p%pN zYrH?3FOr<(>@AWn%uJ`Dp)p;qR9Df`qDrDvO!fVIla`Y^p$=eYXKyqfhOx4>O{P{Y zmGbuPvfXM^Mj+s$Iv9yb>gwu3p#R?3(5a}aOKWas!~ChNoIt1BDhC0NLuN7(eKyw? z2x4Pxoj4qYIT-*7c{s&7n=Ih-)-{NI<9@wGoyBRNN-P>W7LH81^H%`ptGucz>1y*g ztjF1MRodlFx31^&^_9SU{MQ&;YD!A`+x*$U@g*BSR7QpTQ_LN#@DSLyMGOYL)4&qUtrnZKk7Lf>gxC8McT=?jBjtX=Uw?pKSAO@9`@JpZ>G_i4 z-+xVW?z))OyPPeIX)51)&F1poEvYJw?{s_V8yI+9u@`_wN>DQ}r22hc{J4AW{nn>K zJSTrF>*(M)Z$xbw0;BHwJAxaFWI2R1^!549+X7@6J|s+j?5?MmeGxOm|Ezbm-eeIr zl9QLm*!%r?V|KOCLce!>d^|CX`NLbJnVcj9PI~n1@!}}X@1tli-NN7nUbu)oB{}(J zRQW#U7?!!lpnJor)Wzlf>B{*Chdm=)%F^7t6b_d|E-^9Dw9g;-8Wk58caOx=SK>+u z!RgoM<7!SNkL$UbcB5%Tl_4l3LUM3$a4Jgv-Tbe_q@-=uP@j01m@UKvOG7wV>?qZ8 zw#J186=_99_j4av!x~>CMA0!oB*`i$%<%4@KsYitn}$njC5Uu{qq_UHNn~OXuypwY zS&MJSzxrMA+T2j-LfJ>MUY-(!S z4Po5#Cw@tw45djHK8z2UnH)t!n0N336Q4l%IleNH#~jsK|GY*|Q5O z?xn&a9_81f_8l+PP0hiLwVxz8voiEbjqZsp?v2;(yXi)=l>)rq>rvX#Wz(!%$Weg# z%e~#lo+Ri@i895`wv=-Qq)>(^6B z!E>4@#6>?<;x|B8 z48o7s@BRsskTt07{qigW|N1x)bcf&Xha_~tnFQSKyF=nB>c_`GNDwd0{X_~aIOk!M zLwS7K58rlK&J+ycZojYBlWpgv54+7)X&ITwCS0FO6T&3rA_;DHqi-WGV9DrV0uxG?MEfOF`wqQGnMV8lnZ>xwpl;27!__8rURS0AJRk@Lxn52O_`!q56u zeaB&Rh5HaGBHZ0^A#20u)1o+@P1It)vy!w{q(Q&$XFpyRld%|cXkvP;`o6u_^;zGQ zbq}cnSjMhDm+?qLQyHvB$65An9whKfW>8q}vwZi_q?G5|m+MVO8#Z;WIPtTv9wokl z&s)y5Gr|sQ9DsTHRz1yzj#5bEBKi%u%-kow^vV!6JW9%t7-O_u5R9PJ0$B#LNhmE5 zT+8Niy!3D(6S8N1nt|uIx(XCrpg$ae*ClhoU%vy~U5Aw5(7W@hx(&)2zTn==zV>2~E?(u>^%AVAcVkx)l&)<_Mr%gQx=L#q$1; zu0+f`hedhL_71VPE@fB+iRCV43A2=6 z#e)t7`Ggp>v>0^nAsqd1o7-@;dCNTzENRF_-56ZoWR56qX7=L`_{I|s-R0Ihd8XmHfFuLGLG2oOG zh?r1tZ6gb-sPr+;U-N$)5vhgJ)jn1lT`x2^4_}tb1dhTZcJf{#l3W(3i1Bj+IA6a2 z3F@q8*YkpY_1C;ccPO+kn*kd?4&@M-geA4}@f311#=d`bdP)e;504{sOvu-gf_+Lk z8-EvF#HMl74Tj?V3-}GDK)16N^@$+FSN|y@MyPD2`NXbN8Ax!ecnd`*r1??xAhClI z*3QeuNjyi@8!F6DqvX9p@u*_^-JTgnvsjJKAi|Hn6ij>^BZ04%gP#xnd1HHdvJS~# zn64xCp44pMV-X|51D^64C5W2a|U-fR;4q;fbbVS*nJI~n^5Yz?p^XXPlYcyB{iG&}- z#e?6m(Rpkp)x&Gh)Jc?E3FcDCBXXdn^F(Y8P?+w;uz{xG(%EXaCdfR%o(}sV*?}Lz z$8b^eNLU?)yoDB$vL(pkZw1JOe+#$cDMF*;d1CUs_wpbR8-k*qqe5kVy_jKv9miwf zqwOh`9_HQlW95v zDwo7aL5RCyvRZ7`ww6dx9)aHDoG93s@Hs5EM+2OMIPS%v2b1&fC4U!?h+m}Ki$mJ# znO|8R&vcR#P)ca_V=KyRc3twO4aR_*=X zH*eanX9gBXK`}PQk{rAR7uB94!k#e_ zG{!<*H`L`M3in6+$rte_bOMW74FRWt5J#fN<`b7wO~tc*EOMWg^sk|AzrkH(BFZ3K zGgWBrL+x5=tU6yqeAtRdk#Ztp{ofR<=db665#&Pz`6yyoor~`0or`9TJk6elMPV!? z%1k^RdK@I2A;4NW?hbw?O?fvwzW|M0pzU*I^TS!x`6i8Tky!|D=k{h7|Atcx!QAjn zI)y1bhzFUvIIZ!~W6$1Kl~AyDke!V#kCN#w2`wIvkSDDUQ1IQHzJw<^iQk9fCB)JU zjy1b4?<$MSskzzWJ#b*THHf}U0QFfSvEkrAc5@7}2ajB+8`I#Wzy{G>H*n8y5O`@2Hb}A#~QncV1CrzKXV&;X0=%Qr6aVN9Fv*Gg{6Ynm_eNl+z znw#R+1`B~CSR&pxvxFD+ipDBsUW9BAskd}5PPvPof<2mGisKVo`J4aJanMxW$XsyO z$dWb?>Y{^*1r}He=4jLw?2m*4!RjV=rP@Hm>8qO2?TF?ckG+A>q*%TNr}<)NI&A5| z){hm;Ty&hI z30D!1Y-Qe&85yS!W(RJ3Y8kNAP-LW6D%S-M*2P~?sBXv_z?LCR5@P^rW$(#!DyksI z7XRZ8UIJ?SGlxLq4dgv=%UH;CW*tgN@t6ASC*0Kx&`;DLlG`rbC36sK;qC{2d^jZYn z!%^GCAK!19nqzA}k#NAshMc2g*HVbUJzc_spP^deX?RRU>_;5HgKka)p1CcG3L5qx zp1(xv;P-^c9z3g`3&MuErjSBq#JQ$Yncu@GLWl|8VPl|z7;(^tg1o{b5XoR)3j=O* zjR?>Qa6ugxKw07C54^;~LcH%|n?;ePz=RcwQGW+xN*}V(4fHD>I;;UZz6fr`Q*Ql= z{_tuR4J|;T)DkHS9z}r)$5`;E3r}hmCPPB^5GImqMyON2gZuqQ){}HG5_jQdQCd{O zB$Q0+L0UiGmdI@)578<>P77rK35AYikgTT_3_fCAsacSRdu7anZ!Q!n3bE}^>Kk?h zvUzBn_aW&|Y0#!B%eo~=3Xt@66P1u#Ch2;#BSyd1s7cuC@;t2jjDW!xg|AnC%(gwm zy-h?L-Xc(I*t{8E%y<2inb=~|fwF+ba$;a}iSpfrbYsA`AApx zhwFG?vi?i-4a_~3M+I<`UsB{m0#NaQ$k*?h<6;U#I}XUK1KnmsY;40OUR&~rzc|W|KYTv zsPj#rJELHPvJ%n_&i{2R7%aQ7fn(4CRCwA;0Q1!#4=S!)N|F>gAvY^IXwgty=a9gU z$S+_MNz?#{($(66Kd~^bd@0;42EHcF1$SZ`OgEt;VtB#38upML%3^8{t-SeZLF8x;$h=-okBsPO(SUjrK22aCgcbU;m6)=1p+P- z2C*3Sn%=ut5fOU`qJp7_Mj+bE#rc{|k;tJ76_0p#!mNcqnI4Dn&^(DyTL6e=cJX{ZM1L-h+ez#~p_o0R*5({){*7twBZc_W<782xBg4p~a;q$% znF;OB;2U}G7vv_02f2KL2ML1tZg^Hmw7}KFiXj9>a8N&5e?1Ju5M~DDmMQ_(bN>h| zRv_?MUpJ%XMBquavWV*KYz8tZe=@3v_!z3kaNKBLnBv!8&}*#neu#ffux#X$^v?Z- z*z?8hVVVkW!gJkyn|*h?Q9DY(qF6jC2&ubL7{K_5SfD4JS2y^@eKUr71O`DTetRUN zoy6Fyjr}{B@Y)J7IF5co@L3j?WHQ2C)a4k(Z`0KyOp)|RvKa!57K=>m|MI9 z5_cl}SGPZXQ*yJ3K+gvjd%WAi{yi8(ah8ipEuSP6diu)8=ptEUCF<+P0 zkDay>y@(_}EWrjcR66a1={j2VkEF2`R)>jg6&QrUK(P*8uIo{$&!@oxG% zkb(zZ09fq;$n0qvl@O>=DMsY@7h;}7={j_pE0_DO&u?0x0VB_60N|_8wHy!R|4^3V zv(V&x{TL+t@>iwA;~enJ?fty6|M}QjCs+KwqHFK!L8=H96V#&GWde`N$_HgdRwpOV zx;WTD$yogZ#ILl}r=%dCJ1C3~HMTBU&EW1D^HiA{83)zFD1q{(+kX$U_eK1-Lp-=o zE?oV%a&0xLv19QpsvEk3yx}DDEHVv-jguBi9|n5#ph5bX>GHkZ@9kr#%!!EU&@L6s zzAy#IfNCm4rnf75zs)0X4>4zxj9}EEnL(H~f z-`xWGKgaI7LzHVY+!?$UE?D)mIA-! z1M1;~gvupl`-z2O9_u~ByOM!~Fe2dzw##c2UwW$rdVq`F4G9nPtcJ2h%2x)w2wI;% zlT>D5zAz=ikteZ+Pak;ecz{cep6kv=MUzS?otwbOM9jcgEi5cctoZSCt(^w4S1;FJ zmhg*dK$K3+8-tW2o&#%+%khSm?1o5eK@9w<8@?OsP?4t`F4vZrhK6>w!hryEB4#IV zV#rvT(NGF|A49N5?<_CMG_nE1pUu*Ye?^FrKZqyZ#Agiexgb&D(%j zx6UH~-2)_Q>;#F>XpvaFG>1?5_ZaBWc|=fezrTyJF9GXB99Q{jaF`P1IO-9w^4(R4 zw^HpP`j&T2F|lwh=axgzB9i0m?jK>TB_qFl{8>ju0xjKRXedDg-F!WtoZ^pi&Y@vX z9z~%gozbM$BhH<6^sdloxI zI~%kWb`){aW}oAB)8mTcxT%q=Qo@TZF4%Ul6gE9h>cxLPlcG5QQ12*A?K(=K|6!-QN1M0qQSyf&vE_*}zQxpLt@cDee7Q_?B+ zIOvvGi+GD(fgo zjWEgpx`E3NuzD-wd)TDD0=;w-uYWl#R7jGjZwL{U)CfgNZVJrkFDElVM-a_tMKT=W zzdR+`hsB#{*ny0r0LTh}3qXMA&GdKlF+r12X!O0g@*5Fcwf`qb+@J$%gf(&V$bm#e zptoe#TmG*$teKeNERItlL@gq5u_GDgnCiM+3LQLG0LL6H0F(&@3=lm0Ddv8of6Meu z7(e^}-0RZSl+om06#OF?b)UvVJep7bYe$zRSgz+ewfhEm)@~4)lF5E1X)Gx7|JU+A zVDNryGDv59u#NPM zJHs15&-;)p8A0%DM+5x-jYtkRt$Tj6!p@Eg19{5yiCP%>RIWQc7zN^N;vyi=o`}(} zYmx9ibLP!{<$^NDQKAJ1aC_=pB6Y9xJm#DU(ONwaxVQjWAlr5o%jQNXUU{QyCNe{Erll*7)Y4)UaMhhcX7;yj7! z{d&6yP=ee@zwZ6^U2nEh&4=`|{BHuikcD*bPC_6<(52+I=IX7ND~X=}UxyVW0UhSy z*NFP304Z33wM~a+84r-nji4Q`<>G<>#&QlB%7J~&@h0Vffa%(j^?!zN>i3v#f$k_W zSMq_AKAdOTWjhrz-tRDC^$jyNXZ#m0IZ*YTL%S=uXlg{v{P$C=c0cQ3 zVa9T?|GS4W6c`F>70c)25I%ZBh{?ZJ`%4-EWC<3}2g~o1&K^6BYyJ2&P{81y+JgN< zu49j@KL~N;^G96nr#UCv91leJ*I@ytnZP!J`l1+nsPO90C<2CDCh32Vz5G2G{8)4T zzh^^$ybcNcKK{-zfkd2DR@Ek2=c`cC=hA(p!Dkgx>HIW04wuV$oOp`t%ocMlM&(3lo2kVHd4q~1Q9zvjVdX$ zWHLG-gT+@V)3j_14XsN}d3(4uForb>)FA9>C$O!!nG=N6FAxTH!eeI}UlitX+6 zXuinm(2daX1wj@n!d@0spCQO1e~+>;t1$3YGFT+?ID<~iB4E@r!KlMQ6jN7SJsc4F zj)=P{coz{r$9)h)ACOi7iR8YFJ%K(qe+!4>PZm^YJtN0iKJVoqi5?oliqL$1j0xMi zKaS5lAf1x%$u0hA%!ZsI6WgUY*Gxt}BxTnfs($11M~9tNUevhjtBk>87fCqyrDD+{ z;td~j6ORB4&yL0USZ;i33QpcFehejXg~&2QtdeGx981}4L%~|uoLusuw0{kQ za5i|wODoNT^FU=}R+WweY;DN}BdL{MW z9*seo3fuTIQX6v08flDL+EnZ1`oyK(jH0IE;z|R9U!786?lCN4H-?>-*0s^t@id!H zniVVP2LnF3Ee$Q5G^U0IjayEyLeJMB=pjGTN9ygWt);A_Ln`TqC#^tLOl5sj=ysGY zLz6OOG>RV9aUOT)%0;>41=B_ZYBE*C<%cOq6fCJs+(z1pRz1W#W=mf|G81 zZoC@3U+=;xpB9l$H=x*bV=`m5ktY7axRvsUw5Vuo6P?J9B}7piNxSKE@upQPwB$gL zmwT!!R=P-q%Tvj-6NHB7x|VCH8Pu6Blf7DR(yZ)i^WY6a1f$VbT+x&#G*(Ki*Ja{8 z)zo;|Y~8||@QHGT{*sod=Jq_e^4}s`9$Gj&Z}OZ}w|GcI{#4Q9sgEQ6=rsrndS51d zqT%PErzxqBZqfP-md+n(cD-2Pi0g)fodVqo3`*Ag{s8QnL)nr!)(pN70}er0l8FNw9xo@NU+ z*?RDJ)6i8pdl+F02}|d6=@^iC$yy&(E_zA(vFqF{9YMs%@X%rQ9S0*}w3hZ;OGd`f zyE!t$nR2-awO~u#{Q~pCz(M<(Q18^~EE18f&get1%0?=qEIok!UDeTjS{P+I6pmal z&T0%3GGZWx3Ib=dv!y~Y_*Ogy-}F8{)PJN>K3FN-okXS^CgslGtIJVA;q<8ObC}um zK2{6L?lfYVgRKe=YsBLhSsO4hDWx=BAu%ZJ)>_7bAIW~sR4ZiBW1gEz?M!ie9$&4n zv(T1mOO-Ndad>uH{rhr$y$M=$k&K^O!Ca306%&vWhs?kR!w?bgO=jT(7$W*weU_bWy7*;?w0XMoNIqRR z{Opxuro1|p5Xv=NgjxzZe{9g7M31{kT(7!l1H*ovyYQ)(wdNTwS5m;qr&05uYSgxt zrfFh~hknR#ehgr3v$7hL#!Ssof5Ui~s8)1!w64|oW!BD0oG$>eRblSXxVTXQZPYEk z#aP!ri87i$tz6f}2OUl0;AAQ#d>Qn1Zdu8c$OO)I>;Jjbn?h)mYA$b7Fwj-qPZd6; zz|}7M!d?;Y{CHdKO)Zg43njq7l#(MShY{0f{WXu^U-G>q>>q$QayYuoFoYDVj5)G* z46MwG0IQKUW^*C4MOXtrdZAvlQOs&=%ipho)?S}b@%nDCQ`Kb5pZzxrXf9G~md(#K zLHGF((~4u1Orhwm_tm`o>Ez_8xj1ZZU1EPJPLVWo9kq@y*&>zDrO63vyE~>*!A|yh zdUJE1gXPll-RDmq@Orzmg=J$_ja7|-b4TAKWAD~^#itdN7$QhK!q7ghx9Nn(YG(3> zSM>Z;hj?P5->CbA`UYPRmrfGOEPyBE7oH8zTNhH45vV5m{iW~zhCoKqT+&*4*MSO@ ziV3J=xc)*F7Z*!ONJv-?1&FpQtv(I>SHB_%0UbQ|1%Qf*!jP3lpi;^xC@7Ru0*Z=% zb-WjnVTI7opFb^;06*FM*(pzR*3eM9{lBu?C>8)KkglNM z#$^*>Das>Z(V5{s$)4MSKuMaO2-!DdEW`tj#oPEYpF1Sr^uky@Y} z4xxEpxg#2-p0m*-a(VLK_qiC%ic!!;*BVc1?GZ$K=zj0Uk}Av<=dSwL#!tJRJh+t^ zI6b*VqHI>$q2Fa*mmEmxDWuCST3q&THvX0|&{jR<0CH|!+&c5Pt1E#d|1s+mi8kYw z$GqO{%l1cHKFq@C~X>z^~k^k^bQl3MzY7}A@S@1gU0cUPF@H=6WM4>V8V-;q_l%0?4nc>R1=4}CM z*JK6H4|Fuu6Q+GS{R4gcw|$`3Q#Swu0q;~Epn&dx2f56uGNvlsK$BcWSSEEME~mEVI~cH-eIfy!Bq zm=g+4?{+3 z75>M(Fg=5PeDBQeMf7cxE0o7)be;%B3|#nC*!WJT1h&$aX2e4H;E+hf2Go81N* z0HTSQGZtb4Qv{y6v;z@|WAZuKOlY-$#;6QF>yU#e#0>qKexqp47vUd^!Dca^#EN|- z#vmK9O0m?@Q;APNVQiQAY2&6gk)=Di<=B&5;zBootvH2Nt1-yy+h!GV;P2{N8-f-` zo^9!*s;<6#e4PkDGG~f@G(nJvv8CQRI`9D|B0%yGf$d86BK>{Y1)>aWWVX_EHLD@R zG9qwQD^*wF@7oIxiIe-064I;WAbM!Gi3aRD`C>!pvUKTPE;_lk^V(fLGM3_=G8C(f z!j^dJSw+8rRyjo&^i=ZGpq9>GWx_IIXiNF_ViqQ8+jnC{uRW>T9g~0hD(n9SWRLUf z&YM(lSd6hovq_Li`uOzgjthqNy-?ht=-q(y^+75L!>|o!E=Lr2aN`Na%7yz5n+L6} z%+wZ9J+|?N_H1l{V$H;dRq=E^zIy%7H&_r1>WBhyN8Y;CY6Amxx};Fk@+UM}UN%1S ztGPa{Z?)1%RE%?^**X*BaH*GSyp#Q)0M80w4(e2fd`Cw{efLy|?qLXBw}0ycK&G~7 zaxw~Ezb}_C?A-__FlhTO_xMwqSmL?fhnppfAHh5niILhf;bV~Y^V?t$Tiju z29HM>tob3kHp|&9*_P%&VvCgmGK7yz!4^Ai4`#P(DU4_2%wWHlobD8 zp8by}x*V9C3Empu0N{Bo{v1XIhOHs|uaSDz|A>SnBrIm*UG+RV^sS{%hWK zT;KdpBN#SccRww?LuppU{f6YtZ{L=)uHK6M9OtQ4I^V+|1>D{_dNlPHoTi?>*TJmu>l3yaY zo&gU>g)F;h5ou}E6ujZP?s5HjWiLwfhP2jT5>{={&BL|_pl9B%eorF$y@(pQnrotU zGfPT5NGEV7Id7YVY|-YW&xuY{;clz*Y;C*Vba_)#!S?>HtU9C>mbMJvp&r41+7cg8 z7qapMK3G3!G;KFf?(S{YXsZ|;LONA|a@f(;wdesyL0zqMF^h?yby_)>uK?hb?VKM? zBj%IE1=x2l3!m4C!rh}vG7kuEmhp$dTHGdTrD_-O^;;eq^K(>N!Xx@m=rYX`GV@B2 zsUxux|F|-q#2Ki%*6VnKldV6240S&fbP6bZDl&zzp{lnUq%J%q%8isc;M4 z4(gAY`sj_qbpX_Ho`HQcz_8NxW=2_=CARLHFR?S%tiDPk>uv&kk^R6WdA<~(D$Qj4 zutYn2-~yFUC;DZX2&fWTJ(^XBm`h4Zq`kep*^XG5YEEdF=hx=*-3ZE4>AK4V7#Os} z{JB-s7f-Ll1-;}_J3-!lc%oVt?F908dB+Noe2nxs&$P*L*Z4JlyjA4L5QLd-+`_e+ zTIW`)8Y}59A%mwFt9`3+JG-PNpD`s~|MlLmQFjuYl?UoCKGvv*v~oRNA(+GJC> zq*{CEK5MiS>xpOFtXE8$;Jbq##OC60UFu5y4aL7&8nxW4K4R%u`4nKL3-5@ zYstTTv|j9ak~}4`Q;elS1U&Lf%gw7;HDxA{#S?JDf1YPAol2u)r`#<<*#QX_;7PZ& zL<1}%w}Xh20N=7GzxZNQTN~Evb`o!Ly0uPrBVt$4S>`SlE6K=5MLxq>Wmy*LL`jvm_S&k&zpi#%XQ@_f ztX{+Ly$G-R&_C_{=MVeoKAxl~fSq31cLdVuev>*eF(HIF|K#v_eTSh!a#BdjbfX?ll3}WroW>}>{qP)aKb2oyry=X6lu;GrZjcXC2=_Z?-soUwUuV6f zI9QE(9>Afv4vuCwn@_hEmI}9)Lugb6YtpI6oj{Pg7g!$wDwuHA&uH{VKsT(c?fxc5u*N1hpcn6| z%lN@8U`3TeifFctism9P5ab^iQ!;tGv=F2fd|#jK2w7R0v)y-mEsJlM*W;pIMV6>Q z==8k6${~kLfWx<{@E>~_s-yvbtyX~#=`D+!N~Tr3Dd|>g2f%YJh(ikQ8mQi*%7#NT zGMG3F0Xf(7|bl#R^bUyJC`tPG<#bHXE3L|A-vbs3rPJiA&=;l87ly>hKmq%z_r{X z@gP5c7wKqJyJc`~F4rJ!PJ^w-MlR`Aas0U-Ng~OE>H@{Bw=F7^4#9C#{($xm(d{CuS$qOr%iEZj@l!7e3WDj+wxOw4HEM*CB}=!%=8=WtS%*bM99oj zC(Hz~r$cdPLvKTl;JiczelDeQ{y}pSiJyueHzj@#c(6|t#x;;5G+66u8&rm@H(%mz z0X5T0+Rm0P&QMCbD5Xb=_PBydj$`i6Jb?|u=G+3>A^Dl+ zE*!yJrJr7=pGwV>6JVa*p-Vw-Dhl$@v^W~WlrJlQUmZ`d0xarp!ny;#+%i|9X$lu- z%|vX(myu0NBidz09lDd=N_Ax1it<1nJnh0b^IOLXv(B&UWncP`C1)k1LkJO181#Yo zynq*5`+QQ;99~W zQf4MOs?`jp(zjP74~f$$Jbv;)MoJuvl&su18Lw+D_O6TF44z99tu{yd5;aSE*;!uI zmilFDO6xf`5(fM?`8YwHgKjNaffxv{wYi@0M($o6gvxDNRb22h{zO&2-)~t!oeBAS z`NVS-bGHdQJkWi?gO&K`mEh=Sk3#Z^xi}Y>?#Zs{`P#~m`wx(AL+HE{`fXFcu2+cf zxt60VowE+Xmy;)4jX?)^bSW91%Iz))Wf7!e-1vN)Q4T+4)QVDUyS5RXbc=E3vv@8x zB`=0*G~rWaWKmBMu`ZEO|Zw)x0(n}Wq^&DdHi5d()VWd1an z_ob#0WZArk;12RcB6k9~<;TVnNJUj~)041S&7t0p3dPgUf7HR_xbmp1T+z8t0)4QW zcLBr&o|uSSHr>%Xa2@2R=6czdRpBdxDzHYyEAE&u?E)S6J=&JSWW3xbMGY2_ZAm7V z-u4o`0o8hr$%^L?si2abwQn9eognyz>h4r+4* zpCVW2Yl&}RiUt{^<)1a0tmKPKsA{OjMm3x&Y3dxS5$^GBw0LugL>c-Wa;DkVJ!@AS zmD|=$Za9{mHRR%9KjVjGX=(Ic|Mti&Dwn%LXOZJ$26A*N<|%)z0cm$MsPm!@1e+pQ z^UNnQljrvfSC1_jWn5?NksgYN$@nI~>&%pd-4E?9A1u|sU#@TyZM2CZ8acVX=@dj@MokG9Ilo-TDmeh&W%s>K)aDzGRF3uH>S z8_X58kI1JGER}_}{7|bmtL(N_A!T(7mmwDkg=;fwt#13shs6xMKX1|ee6C{7#Aan9 zr@@wmkz#DHU73JrFAOk3s+%tubhgKAgOMw;TDZl$$fPpg){0-p>#>H(G!G^Rzh9+k zs1&53830i;2i7%N_~3Z^F53li5AAd0fwNwc26wnBFgwF_G-YTQ|BEN$P`I0bN&t!e z?NJrE-H2%ZHZwWx7W}7$u*;+FI_)c9<#@PzwU3%{VhLLCgOE?~gZraZoAg?^BXcmo zf3BQLaqF5Mqhfof{Nh|qm**0C)2Nu~K~X8Nbg{@+B8m@S;JFb@CJ4-3=G9T4*7LIgH24_5%L33Cj1>DSh?yY3x z0b6kTUZpzr0SbdfDOdB4wyZ9aedw%S73O;MjO8Pr8=1xyZ9N*8(~X?0nv|4S#&t_A zVsNtw?z1Tb%J~SqmMeuzk5LOn{AjL4s z?w9}|3lN|=ZqRQSGJboDF@hjLOoGh?LL63L6q5~;4HQ0-D-2$xpxi=C*oJ8XgtW)l3Zh!?k>`go11v39>YWAHqL!()^|vj0&`4LNAv06P zeb;3h*sQ_0E>Xx?_3ORkUYDwus}HTxY^`Zaa`ozAt59S~b)7)DnhU0FidsmhgZEz^ z5kJFqv=_5SX&xt$etw;kSojU)o`tkH1-U-`j6-X{@+;gX2U9(gWF`1Wg6H_6UKh^R|Hy0^pju)HX7ll_st zESyaGH%QVja*diyNlX{_y3>l|#$gF1qK=oc!6YZTUHDZQ^?B~zLIW|}gO`_&0FOml z6hj9UTLwT&L1F-G7yS==cYuJH$p}^b)*Q3?RM~nXvnO;>{mX+qr&ABwn%i!-v_}9^ z&}^1wLs2&v?Z9Q&oW8P$?|URgzpxLA{7Rhb9<*Km=I}v3y8(sDt5#fZ0Hcd7i&q~a zo>boy?7aJrdjI-*?uD&PjUb*&K`vLnGHGYXFkBmQxKu-ZnqZzXO>MipbxO7D_pi#r zpw}VOH1IGpO#GxfSUY{S&*dWRe3vSL@|J*5sdyru}6t|hflviS)nBPTB_wgy9a>x;2pe7+(~k2VW8A!(VwzKh&o`3uU*ri zpc~@Da_rk0p8~QnE`@pz0CY(od2DLRb}<$p^Fc#HcbqXfBxAO4IsO5Qn&%=p_)5Fq zuAZ!J;O-D0S8ctin-~8@etin3u|Pq~UGE;5(l+hY5PPb6O$FyOI$^&u-iCeVERg@?b`DJZXfC8|YPTdj%U|BNK({&dJ3B>?sOuwpGq1JMXyjd7_%7y__&KJgC zxB|vYV^-jtV!1TIp#1Ti-Q82MgbX67SpVuC__Nh?ln0kb^E{UW6lw9uS$f@Bzdaot8#5SyyVdYLIXMYq*hf%{`nLn_-_E=u?%e>ZRkv#d z?;tXTMg2d6u9@w6SF+W1MprPS%6bA#Wt-uX5ovTCS*+xmGYbbSM9?#fJ+_{TjZB4|wI{o^TMs zBQo21`t_p?7`8JZRO@a`U5b^99b-VFzw~@G?A@(_n;6k6jb2T7L0Byj;RC3I|0GBf zB>dYxS5N`JlINp4jian^irQOqZqd@DU|g=piY11lWz*n8giTu@>I( z0-m>6kXz1dsc zs~so{bs$tXR?(;v%#9ZF_eL1jlQ6?t;_c`gj|R{^+&i*r?^h|B0+Ih%TY}m3QZ9?n zuTlimX_{-G*6$8>kN_N2_i5(jesPe=rPiRS9E=0D9Qk*byd<1HU>aaWvN%(1haP@CAQIZ-x}0a7?dBtq8E?5m-3^Uy%7 zZOl^$mH(cHKk-~EEFl1A0NQaIFaJq69ESmm8qNmoFur|_A8Dw8d2PxCe z$`RP5m+=jnXr&yxC!!iCH><;1I9DgO7jou;Q1@)i^Hj0X=y{c-ZS}79x|4(onQshF z(r7A7tlkk7!c5GuZ<}y;EFN2G@#t-#HoYc^kHM^ku|}-^tzE9*? zQB=Hg=55c5n(c*`=?B43(w=n317q2lbLQ}@{U-Cw(&^nI~{6`}>S z*sM+*PiOs~I?ggIs^@?Il!SChrD-7X7Tg&^8eufkUbY?XZFmQdB^L%H^TXcF$bmSc?~^xauyxxk@kY7wz=+jxjHek z%HGA9Cf&tl9V|4NW}6PEN%PH;lrPk2_ZTbZ4R$?y+5By+vVL(4`Rr=FBH#zqYFNw6 z^vNa*Gl!j2*$Z~39s|d9_MYrg?T>5XFNB_&%5G7od)ShfX;HX`*p9(R1pk4Mn<+0X-Rtz(Ny`WDQlGeX4^~r*U zKRKP_S{*u0@g01ka;iC-Mxq3Y8ggT zK~S%EAiZ6#SoZ>Eo0l7cD#5Z zxH$OBbaNZ0{&F*AGL}%hfxbmDVuxz^dyq;JUk?A6nMvA5%(@>m!}er zp;J_uNdPtuqV%5Fe=vYojQ6dEsgp!pHXDtoeTg2#n&~(kulN1aLs$XE5wWe;8$6j( zfSwf*5y|C+fi55dfo#SsHmE=TZnXir9tT+S8Y+K#E_fTDpJQ3+%$g;@sV1{RBK4Xo zY%Ut0!uqt|QvdLfaw`=B+%ZRAdxFu46;+zOF5=hMH>sY>_{{wo^b#WqbqV%JAcUUx z=h1t=SYRIj`=>2}LFfH=$^9$tDrFu$Ru!-Pa?7Hfij?&%KWN%)6QG^%&rffvf)(L9 zZ(j?#Aj+al$fO}R-&{ht(o|bB?2XNoySuw9G@@if6Z#Xu?*e}LE`hb#ZZ9S$=S}P9 z8KYy@9o?$>Mi5fj>68kFpVuOOc_tL#O=5Gxk_(pqR+^Xj_rq11Bs}ba8?J&B4bc&_ z>hT5ARsA%%UzkF`1#W4!gtCF_*}5MI$F=qmOU^E5{)-YwBi~s*deO$ zAbNC*nY5RBXuuF-u1S}Ew0mOGIi^t8GF^mSzw8Srx z2BMD_Ep&ls>Qq_|MI*deg11GNu;C7h z#ulBFL;)I0bC+>t-f}Fe$t1fwvMnK+L)MR>(VboAk*b&4??UJccNqCCzO@NJrn|$b zf}{k5f;>Uv;)l4-b#DlRilyxbq3frix+R~neTmL!e2KKFzDhXgD=TS)o9A>voYx5J z9OLqj2)UgyZ&j1FchgbQy$CJSwS4nIL{wS+O!?)Ahc0xH{zVaKWm(;nAqBjHp+)5^ z9c0v*G0=3~h3uWt;+NNT42F8i57W2gnR6q&3bWO9KAsLiPQT35GB~yqy6+a`tqZqa zs#-Szm}V>852~T5nPb%In@OuaflWCX3595AOwwa0?J==RK-K7y7#6a+gXBBm^u-05DG`LzD{6iNxrvYRnv#)yj?^(0y|WzxsWlkZskxEjx91G+mMgjmVS zX|{3)x_%UeC3ZSo_(Xw{^nn3C8XHQ#5bl(|pqUzC8VzR+?R>3vtgPqCC2E zsc-uze-h9w3snIucZg(0x7L#<41?8Fwa_&=$WZWV@?#6}gW8e{tTXohM{ z3qky7;LOaO{e|{9$Jg^PjT}l$%J$%-qAqcDcjcpFKm4#kg?@ZCHa|Z=T`s^Ddf%kpRN!3$mCqsKZ990|f&R|6`c3D0A3t}KPOIK(8R1p6M$ zW@Xo&NmfT|;1?93N@$}9{=}i7*yOb<5xHJyt^%<)aUjxp4z$@5)=jFYl3A+W9_C)n zB}IGs19PZIs;_OxR`ATRgUS9m+@sl6bN+l<(ZeE3;PCOSSL4A}{UN{l3X!{%(+6c| z8|d`4D;r*;cM}js;P6}EYr$Hr) zL=5y`Mza53d)pKEqn`-Z$l5A6D*YF4`jhhGDq8f1*gz`*g&RkD+ zRY+B*I#*-R6cO+}8``D0b3A?NrM;IQ|D?gre--MNx@5(&AEzx>0SS&LFKaed=!dAs z+R+Hz4eSU{XZL%jf+X>zI)%Pq%*4jRJ^)82zz>C1D-r2|z2w_yqY^K&^|G_dwYTV# zhf3>9pq4PWAW3uk?n;+qRHj1-9tMblWBl6?hG37bSMg=C@U&zQ#y5Sc!i!A6z2!-M z5AoTX%)wu4lvc!$F*YB&yV|1R1;q~0ya8h$d~9;&ysRCqiPb_M?;{s{VNfm~o>HZW zDy_6QYjC>4OhX=-zj)GP$S)4%io@bxZ5qd$Em9nL@D_@uLFe+ z%&dDyt3(`vVh3E0zA|ebe~_>)!c*-jR4vR?J9>0}&294-`Vwx~@Sf>9@|P;PwJceY zQ7XS9zhj4>p>dE2-DfC_%fSzlbAR5-6w_4>4UqR>Q!X7UXd+IE65XGy-7j*=Zu`L; zDECHra(igv0`yoKbnY$98YD*ZjL9VmXk_bDrv^h8w@5A1a?ntB#82e??fS~xl+lz5 zcNZJdP_C&RzIuh(nx|6cmrLyHQawra<~Qdnt}QwST< z*ITz!!I4NJY_eLSRd~C#9d7i_Ax|%Zk}Ak6o|yBgA(Na^thH2h(KR(T+T2^hBq5!^d1d z<2V_FEd_pyJ~B#dHt#M=pKDEYTF7-1B9_T0x=WDH`!c7~l(U=LHHgwz;aqi6UUyFb zHTOUdrOi0FgyyjS5p;gJm{-pH_0GBdm0RPj=sJ-iM5Iro7aT55{7hP!}WNo zUCnkH$8i0LgoOChsp6m9&pxe?hdfUuqN#Mik@H#Ll(qOt_H3Oy^$V!|0BxLv*Lkc| z06I(_Bmxi!q+vMEL&`LNg!mL?-^vJ5qHo;$Ms^D8visXb@@4k-pxxi+%L0Q_+3U+R z1H#F=kq;WZk=W!KO0Z7$u5ShJ{j80H*Z2tG?>*zW)uscRs`qwD()V5cTK+<2MV9e! z72ogajhrdi8B`c^02RhGID+K|Es!A%5-zGq5X0RGWX(vC4Z(FNFSWJWj|oMSv!6Zh0%Rx+4b+F0KFaN4hNfj0Ri%>~mBJ2w9 zQ@}u$GL$Bv|Koh0>i>s^0oOn%D7TXQ!%>6$BRiMH`vRo=C5DYUZ^KgXd7rDVZ%*H& z5*Ke?Q?xE)+iv!NKj&a=z~(T!>AE*a zmCg9vsWlHSLVoC8hPmLemFf(nZGoQnOf2uF_SwmxLAeVz_}IkV*EC+M?Xb)pa0&0W z-$tD7Uv^8B7)aaJG7kHdBz&hjZ-%2LNzcF_6&-VCI!D0!^%ds*^BM5DD87FZzm<|( z4J{STUH{IBRzJU2a_%8?UUoMbRU<*7xJLyJt@s0haKMPwww!UzbCCU|#Sh1ndHN(S z7KkZT=y{!}k#*7jJeyGp%wqjoHDHKXJ(__Ux})&uM^w|XnmG&NL`o_Ds`z)IS| z1c(ojbQ7wX{ksop#An2AQn$G<-~o4tY+?!IbfKY@i5eOjHyrwn><@;-@7&XDu;PzF zyRC82UyY4?f*7qK98#DxhVuX% zi|+XeQ8h5etRh3U+yKch+075SDZFm}k9c->Z!f2yV8=YWXH5Vs9Ze);9iQ5lWs(^Y zJ0xvg6$l9^DpbaUve%Zefu*XcXS?`!{%|^vdN{dIAE5 z%`^e{CNQ#5)~mMF{^#F~tspWoG6Exy#hrP&b)MA&wuYP*8lTjnT}F6qZ`3@ z_YDkS|MNsuN`W0By+8=4V>RrR6637=6;m&CYU%2dQM%Ie-XSGb0>X{I_q4i8@E~?? z5R=R9=9OVlNgQRt1l_Ol1UFjEl)71E@Ub-p%CwPRvw_dqWF`Y*+qxWj(LYh3`FCzW zm%I3Cnak*EZdy73FsiE^!OE7FmVdv*i(sb8CI(@I3TjeB<6N~1cguj^<=^XI5Q-{Z z=ij0V3k!$Cr*FVkO+e>%%C0T|idVolSfu6;U+ap70z|zvr+d>p2z4!`xcjH{TWJle*srG#*~K1Q=Rd|8D_EuE5Bkk&W;MjtL8TUa&XlvU70!I}fn9 zEG~~Xz&P7y)>{KJiL}wt)^Sa#_mf}U2yc}8!687F94u+NyWHBEotqo(0$0k&*G#C*w$LqWaJKn}JGDcYQoy z(I3l}$pGp~R%sr&r|?Cwf6mzKygcPUYNSeMu;s>^bOcd|z}o*5Atf-q_w<*E+*!T} z<${%3ke3#nD)#DMIkts`g~sE@1V0}B7aUYts{X%AB_bk%=o7qxf7}Fdmb9@2B3bd` zQVQG?=n(>E)!(-Xm=ejnw|ZHSf)rjdwQ@jx(&ULx)An_ zz)csTEGqww8rT!6yYauG5_X6U3fdgM=iv*l-|7br0Id18igtFz;pDv1mV=25*ks(X z!2GyzEASC`m0!I@ZVV3{7?2|~Wb(N>2cjI0>}n5XJ=>d#YhGj^QmIJ3cFc8 z0C^}6xR548UruUQ7=M04IvFFL>NaP?=X>q0S8fV>?VkWlK|62*M8*?$9g!^HJ*YKp!$~$q80CL%^Hl%>Pw61jv?8w#C)x)?p;(NuF z25C#aSM~)31x_QMci(c9M#gdI)=-;K5)*&Dzry)pDgn>glr5Q0AJ_jdfXMGG{k{iOb=)As_#|L1Fd^QDv zo^%5u;D|r|il%*krrE2O17W*NhMs8XG9(DfwLHcv7T`Q^29@1`cQj%KN#C6}j&=Iu zyomZVH8o?x!j98`bka-BgVD*!i4PdOuF}x(!<#?242!@oiPgowXk4@z|2W;@b6klD6+(~VD-5Z=Op0n64_E%1%?^}1tvFsb*uo0mBW(7EWEPjvpBvNFgSz{I&u zR$2*B_1ABwH!j{4phP-;;5+hJaqI2Vy^>Az0%;)7&h7>c*=@$CzU52t)Y zCT@9D5YsOP3>*}wBi~|>Uu4r&{M>r#Q(oL`jhP%U9YDkIo9zu_Cs9T^%Hx?M*itACB+`=1@J1_?~whbz?6neM>=+kzDd# z;0PsQV9LE-GBji%Of*RtQ(5d0-LFH8ix0o+l+zv{f)j;bEgk+VSX&@AwAPbv_b{A> zdX;eWAiT7>Y73>QwV;aRIP{NzLs1gB{G@uc?^rCPXpi)$iv4{+}r zx7nVM^&&($^BTTnQ1{G^E#Qxq(DYHkPO?V8RFvN_!1~^cK+y{zu$nLl^a`Sax+{qO z22o{|YlHEPvF)Kd9vkBMLkvS)@sXp{M7w&cH0k$c}QEP(^zVB3IA zcP;NjQ@f;B9kS-fU$n;vFEu@dp z1kuPCwLsX#^U0ZDA%fHQuL)xm+FKBZ#sIx>=Y+-Eb|S?7NDGYQ-v|1DeN{3+YUMX- Y$lMZ_$Rn*7V1SPVkyDc`l{S9!e>_TP9{>OV literal 0 HcmV?d00001 diff --git a/src/main/site/resources/images/backup-cloud-appliance.png b/src/main/site/resources/images/backup-cloud-appliance.png new file mode 100644 index 0000000000000000000000000000000000000000..76b6d5a7964b64d8de841db6e44cbcd48efac3f7 GIT binary patch literal 30114 zcmb@tbyU-D{5QNYU?4FTwqkygMFBScCGQRxt*q|EEbc-8y8a7}e3iaYo20000A73CLi005xz?dL8M z;@fAoc{mOL0O4#D6f{*76gV_p94&3^EC2wBsHk_dj?-`N!onR5^XXyP715M*k!A0+ z4eFWi{puiAN%+1Yf3Lo8@^SuOcZYl06Jj+Z_V2Pz2pq}JK#1R9wph6QJ@U4b-#fr6 z6>T}XF|K@Rw<%fcgpU>*DRbeunJ1}~d6&I70 znk@PLKIwoQ&yj3lfscBFp#XtEN3ha(tI7SAIdSmR){U$C`{F_nSEjt3jjW?yQNRp4Qm?gx1C(vwzE7Iuh)_9m}ca%BEQ_2WhEwkuKLb35`w zmcVLG4gSr04ZYr49y{MY4C^*%Xx~C;!j+Uvc^^?LCgU@VZZD!Nr%PQ?b@nb^{W3p7 z(rp{WBcy{EsONu-$j7a5$|hO5mY9V3Y(-TJHVf_YzgpGzr{Bus-c@p%AL-}`^Pvh@ zpV}lG(9xi4xI+hpU!T*NzoqrK!j@&KALk5=mi|b#XkvZg%NWi5G~2Y5d>5))4iqVh4JAjj zI%#QxK(7)tN8cneh8KRIwAk@lL9{xzrBakke0}!PW?+GO_cKeH8*T7*=NiYOc!iIt zUl6b6xE$kJxgv+DU7SX0SRx0^X8#=>$~+z9TVf6Bt}l-zl3r zSjG-X$-ImWW$58BZng0B9dx<@d3k)PbsPSD!u@~nfuhjL3yv#TV{C&H5GR>)RR zE}jM{(;g~kZ_nPJX`QK`8J{tIT!?qlJ5#B9;LB2zt5FC2*jL>vx$?`C3#nN?2?Wmd9>F*_j9>5t+Q@s!Z0LTJVUdX+L z1Gk&N>BdU#rvaWi!#_?f^^AV6gfQ3C6UDq+^Q7nfUX$m}r}@3pUDxHIF~6EF`njat^Yua-tNz|}$-~COUe~Nf zz4t0B$Zk*)_%Y-yTalEeSN4JI|K}Gx-q|@k9-AavBOD!~!7W#75yWiu!+?SQ^ z@kGpjCoiZ#nldX-*O;6%L(2D}TV$ptsMdyL%E7uYx-HvjsJ#u)d25GgDoW%yBnT** zc49$mO=Lv3HMM4$ebtxUyY}%^)mgp0dP>DGi-L$>)?~9*WJqxQ7+S3JS9BT3pv@gR z0J1X0(_sm)@^OG&Vi+L=NyX55*b<51#gS-QiJcjRk)4iVgR!4Xr)iQG1N~vClxPwM zP+<^^Kw!Dft8slI&?IOc@k=)D#XO}tDBgCRw(lFp5GN>W_V)A`OC!Xl%n^xk#|9ws z=f6sk1*ZV~WN0R1d|D5Edr3SZfDhd?!6N_a6~gmTk-3?5BaX1PoEjE73!e(}vY)^81KtUQ^Xs_a%_ey8p8pA;W|s#A(}wX8=#MJ6TKce=;WeWL;|eU|^bZ zZ@w09)24SAG471$<-2#vz~e(FP^zo=Y<0pW6_&l9ErKnPwyb++YN1)5I%=q9G)HA2q6S-U#^G29oF4zQ2J27#hQ!(R>e`gY^n>jwRRzvL} zFDMVAh$5Iu)>SB3;^Dfuj?gdINhdnGP^p&stCs=Be55j0wU5j21HkhxiIl^2q^D14 zmA*t-bBFeTjvxM6__SwTv9-vk&0O;0DSHWF42a>{OLq8;BhML&6LkQ@jv!e@Suvi& zwi&*Y3Ce_62GR>L`?61FiqN50pAg3{te8GJ(r$4mRSC%BpL3D^}QQ%+syItzm0t$l$jf&{z$v(7ifyr+PRVsbY zY$HZWQQrT4Im8?};dK9uzPwZ>o%pEL03(ZonMz3zJt2-EP-MN>H*}KGT>FrXmMP{L z$i-aSl+jHpIcoCBaVLSjGr)67G2CPf30yc*Z2vNs5IG^RPWMY>ICf8#Du%4+Rf#LN zQ=IBd$hfn(D}@9bFAatmtM0-yNm2^VlY0zO_9d!9vIpq|-tIk8JctPMTO6?R*wg@m z{K}E!nJ`Id=SVL>ZwM~A(ZEVM0m~h&fPa;~QaS)PRe0kEd=M{0NRNo38(-mL&j&CK zCd3M|kH?eF@Id>gNEqQy#C00@D9#)YO(nQXY9knX)?_0wXKzYdN0++R=KvlYa0GW} z0gt)LlrVVCbM712SO%Sbg4k*`vf&5yg5jBp0!obQFLFDoUu_-&>b8^ds8VpK{?1j`S9y z4HH)z*JVh#;vSX*eMx5sp|lKF3YZ4@H|V?MAR{`4l2~4CXK<0p-L=dYT*ACH+(>?q zAixT_Asg?@%}6#!FdhW^@!f#**QtUj>Zc1df@WG8D~b`G2X-I?5s`?Am(wS%lxOUw z^LqqkM5|ZiB}At|DS=3iwtYcD<7P!HsneraH$ec5*o1Hb=$T?%>x$Ufa4nk#nX;x5 z?-EubSEd9a&dD%3$R%oVBj^V}zVIOd4T@s_nUJ>2LE5RPqBCkEI7({}?J{|z-7`_nKG~b={@1l7S zpK!g!{PpwH?=scTE71~mq%0h5`Px+w< z;SI_7m=oCoGcr={h8;XdY?yrmpSUS86q?(%X~Aihijc`>jnGpNuHEE>8!AnR!_Nod)j9j zcZ%aNV$nF;7Ed|a!I!Xd!1{y6refUlTbJ z7?Y9i>0sZ<%EJkJ!HQqiLfSkYIdv|Zu~N)Y8cM}S*AyZ`2w!YPct;#7Qk6NY6u;t% z=)sVh0%Bj35>8`05s?OsTJuKv8ywWR^^@Z)*^p}hob#MXbMIWX5$|Tqf64XXD8(9m zSaq3#UBPByB+3bu|Ln$c-<`&V95cf2-g4i++LjkMD96GqkVT=2`o7);cV8s@^W%S~ ze`POjC9u*#+_H>;8ts3`cnZg|Gf@C+-{(=Y8FRegPbz?)%7_#KGXabjZw@@^EL6_X8V$qB`6)QJo~&^N_A4 zS$S_i{=>)eo|!ZSIg-#$OA~rz0O6cOq6$_&d#9G*!uVlO*CiD$q@tF-lzY@9z6}QY zTge&sagPwM(QMlzvvE@f4A*p4;<+yX-s~qbW?wP^k01_22JcioC+6KFz%Qm?m}3qm zWSiK($<9$!eRpI7(PU-8doM&7NkW3yk+WDWoU{{zE^vEKP1ca%sswV)tUQ@Tj34o! zD+{gB2?S$;Fcm5dFk)@wfm}AsAIP=v2>afYeHY{j*fE6k6DF}dguvH+F5}>s@V!9y z`zor;7(;CSjgAworeV1{o?G^h^H4pb3|T4gc-i@mC5WL`tmKB4Lrxlam$ajO%M3-K z{LTCQ5pO9xk|j1CE{eL~b&K?Boz@fFUZ;HF;x1an10A$YODF z4~*}dO}QTS#N*PiYM2TSuFZKOE-iQaLpGcd&c?Jpw|TqK|6oeljtp}Hks)Qt8zw)> z@vCEn(=w7D^*Lf@F%5Gcxt<={?HS^HalB^|<^52c0@gi>InaNO^eFCn*s+;lYPhgJ zWj*q0wDTknR-QT0Dy+P0-1@Vn_V#9<1>9jdmCBxkbY@emrOWt}*bzi)r!^Da0FE^N z6BpHBv(P01_d+cVL(NuXWcJU+T-G~PMt2#^O$G~fHRH#lckUJBQf%4s;2o@?icMnO zo7&Y1L&@gD4Ds_hVILj+{HEe2lf_w&BWadib&hKppY1j=#_CY-^8_QzSf@`d z=@;DxYws6~3BN7TMzJNgABTJ0wb|k=7;}}fG5InL(<)JTe>bQb*e;yxaeR7ES>9e**aYBYb$+r_!uybs5)FN)E)8KJ zb*-73jIO&>@KT9(G;W|d%e92qBB!v3&pZs-%^tvHM=v!ER8qxrjb0WZv!51Rx9{Iwt16b-g#G{f;e+WhoM zcIF=ON&9+jP%7jFq)~!h5EO%CZW$3?6LLlLavU4tT+8X}GOYr2LtprmrN5eBC)fii zE}r~)H4m%>@C0+=UY8TkfVhNPl@L&xx`f5A^rTBcvM%vqr9|Q&NAh_h^>4df(^@BG zkaUuxAjjhyr9lXbI#blX1`>?c|R3`G&-#ef`z zZbhi0eCM}ymEkoiD5n-B2)%-SWjG`M=Dp4v19GlRdyH6T5DW}``9>EgUs4#I)7ZL5 zIpoV@l$bL6++K&-;`pzOELWy@&?$gtEd|*~)agV7CyPylk@wabeQ2yPH<@X_xPytc ze9(#v;5KDA6La#k2`Krg%d=_wQ$pQ67iB&FJ69QZ1HP0gZ^{Zh#;BBL1^b;*4wfz* zzGO1X+zE^QEp_z?y+koD`Ox!U8Sy-zSl2S+x3I3I|GKrPebz^AcPXI^-?CFf91*tK z{Wa8T9>P^0Lol(~h;hIPn_e5V__WaAK!b5l->kC;rsX+bjZ*h7{NA!+7Xa*5#emlk zSwV0ABzsj{&$blOK!r^rT)V67490bK37C;tP=%5$_;#7O>clUL2oHr$;dZIoC;6Ps zMYJpJGONdVK_5!^J}^FZbxCYDeHCeewJaz7T!*{4_X@=K>1%u_gC#T;*^jJ}MyAP7(r1&WO9)SM7&RQ^qV;*Qf5Y@aEXv4K1F`s#i!X)j{6+&~ zg}kbY{tdXKJJZSTKSf+snh57?kKDzPgsDIS&s1dRe-Ypfny1&07@2wSQIkvnJtH8K zf@RuU=x!>EsSd{u!jGIUw8>CR&It+8PNIwQT_JG(nNeRl!>l5orpwQ(4F}$EigdCp z!dk#eBeoCwV32x0x?pB&O@E5O5K?NfR>8GrI7yr?d9tF8trk@6U5e4dy~lwuGMy~k z-vXd+>f3}?faz3Fx?umgrIScGy8>d#Q@!3H8L;--$w5!2LU8<@#!$x-ArLvi$<_UG z5?xObr`~?gPU$;Oh-yAG-j5!0nTXoLQ=78O_j>Ns38c?!&8bSIDBPbb`nlwe7$$l8 zDU<9evHDYnY3IRQ;5v366In;io2jeB=6+}?N+29O=A-U!b#Oxrb>=gtg zSQR~7PqP6F1j|*Jd|a`Tueo|18((S>y^&%n0Dp9L$LaT#Xt_xZj6gzo56BKAlMQ1S zk-3zvW4JVN5_Td#4SIyZx3)aPI8Qn<^hjcQ5cv)_9zlW#Tcn5f&>F*-2nv}`JGHHd zb(AwK#Iftu8+P0{f&=cLRCv|XHGrMo&jzo3kQ3_75+}?UE6m^9fQJ!9*$avZ^bBMG zmRZ3}o6#iOc^l1uDOc1;izhOnD!yO397w0KN@PkM`=kc>3v(_8kg!S)f&kZ&&>nr6 z#3=fhAa9YNz*B@MLM$Wi)^kkPJK%2sYo~4c_&t&Uf;aon#)1i?0m9>k#QKynTY^0! zpb%jpYc^7P*!%oO6wBhwa=+iJ^EaC>%M3VT>tbd3`OVy&_d!1~w#eMQb=RAkl)pV@Tx)IJtU>YsQ|oWFg9evVwB3M%Y$Uwp6k=&NTf1 zV=SGG98G{wI743R-$5OL_*mqN0uWF1`3*Srl)9YSkkpt0MJ+|(Pk}0(zrEbHa$goc zE#h~ZFv3NSCY{K`mH^~lP;hp9sh2GKHRC0l1kw6oDzhi82|#{){6PwJgK>J9IFcPH z5;r&pE(K%}HWTq6AwZSQD2$ghX&pTpkUkKD%!Nk@67_FA!=_-W56ql+tU*)v$OMS{ zgPXzqUegzRPOwSPLxgtA4N)oC+O!qM2XUxvp!%7n&~Nmb5$7Pp?n|m1VHFerJZXO> zLvBcNw6rEi=HzM`;0W|YP9SuUf`KNWiHs{h=JGN};2N-JCA%1Vg%w2F4j!X#p0vSr z+T}>W&xoE9v`a04UX8W^(_r@u0dB^$gMXnOgIjSxGIxu%6D@RLJ)}w09@J3k0X_keod^~1daq`m;j((<6U1SUpf@kB~N*j z7d8*`TvKI28q0zgs-rX{S10b5o5-?N5J6F|n|}w^QC@&1M&VAxx|9(v0!(Fu{m*k| z$WYhpAO^rh!QrAe?vbYBRgon13_*I~VG9UrflV_NaGLwq+=+Dcq1H6Cng{EQ${+iK z3?*w$shrXSKV3W;w3{Ha@)N`; zYu-I{pL&uV#C<8kq*}0}HaIm+wCxz?Q3K>Lvsygc*~7ok@@cfS=@h11&ub=9r`%zN zR@gQ3u1=pucPua-{Usix8(NDnpCY}cy<|FL)FosioIqSss$1WfIkZ(X%4gE{rzIzh z4%#kW3fTuF1N?Mu*E~DR$ksR|3(1T){IeBsAI|KwD@B|PIzFzQ*wW9=oHztwDP#K; zuz7E=^cXM*37t$izem)_LH~2c)xdvK0h{Skd&i4${?_`1vJMZ^|XP$uPGp+5tF#qe+vsMfluQ9PP*&JD}AasQu+g@(J@Y#ptuA8xBN!gv}7Hg|~ZehaHPo+H>#Mn;a!2w5c$OB!n$Z)Sv^1oLcKKfc+zwo653&$B81A1f_Y3{D5VZZAz2<+RJ%p z(vlSv0DwAj#=7-`P8L4lII;N^1JMIQNp3ULI!dYZs(M#lBW1>~rhOo1=1w4?8+U9y zR#5iqjK*pJVglsn#yL2j*J6NJ+Fz}rOJ$fC6y5+`;9NLtHV+vH4)5|5TF9IHBHomU zGvul%G-CL>BK!M_6fOy8KOpDyl192e<~q&0|y9LDB6sB>`eiucshu+2&1by`XrP1~DVoI1i)0 zbW_zCEm$JifO?Nn_b1deeeF{Alh+Tk=S@JykX?d=mb8{7Om+7B_PnYS`ygbGz#LO1 zCvb18)UGfv$XNf)5E+aaZe>jb9U&7ZKvC9pv4-V9_rLAau7Hdy1q?V7X>{0<>6czr3eQlCHZ)bfQtw=NNQD2sQdg z;CA+s_SsMer>Q-EQ=lfR-*ZSE725f4SC7jlOo*w^>ZV!{=FfrKPOk>W_#(W5XDHvC4eWb%=-DOGvGAqIdA)+02m+w3I_LW17$G?gnTvqPeR zlyKl+juAD_2szfMn{*?%l6@n%d^Wt=cZUyZk~ImN{(a3@PF01l!Tc@1vmVJycAJ4d z9xRlb>uWm>z8*E1(kIuKQyPwuzu-9vCevVF|4HRuWt-LO2vTTyB1A9YFfF?K;BEvu zi!ubb<6ZfW;BHxyy*e-w;qPRKcNGZ>cR-sv39K_*E&Wkl1O8QQSO=EH$$0K=YOhT$ z-4?Rjw^+A2Y_p?XQJnkG&UB|=GQjjaRTJ#72jz3C425Ee0i&QAI#(|K+Oqg!@BZpC zf{QcCY@@Yy7OKPdv+T%D3=iUPP*{f0?5c;hrk-S*5Z@vEgt#9z@qI}+F<>>&(p37w z)H=XWaFegp^WwXSIV)l3t%hO84rOz zQ%G5kNvhvf|55E|xgDx~F?Zc{J ztDRu`+yT_vwu@eqpdaAdd@~WuH9^Ps{;84w`HEkS!|W6o%_8(^v8I@RWf|h#hh?aW% zG9Q|;lVMf4=6RS}!>`FHihub2+1L5CTAt$h%?VDRjn`Dy>a7`)o(=^!Xj)G`E;L1k z(R7V{Lb`$R@eiHz{&}mkvL6wUGhn^J5oy1^xP6p8V%OHIUflZhf`nFzdU}|eF#Lj$>D+MV8 zJ%YsO;@M{9aspi$swcOOhzO%-?Ry_CzzFbdpI+7f#&MG%FEA{MBs26$hJhz_ragu- zz-|Zj7>}K-{swk(qE689F0kgOboipU>4Wl_+6jXI`c#6;F0+6F0{2%2{sgIaY7ho3 z|1c>ck{&m1Q5(XTiK&0rdZt{1IyH_aNxw{ok*}xWl2#Vf{M>nB39~4Uip@{$FAD*FE4qFO=%o>z&9wI>!%d{?;A7Sc zR#}tR1o#d-zOO1E80%qr{O=z0R^mS=+W931Q`;o z2V?_JiQZaClg=%vqWnlUTo^W9O@ImTm(pwkYKK7pX*s^D_`ZVCi+lrUMF+o)x$nr8 z|LmI+mE_d*XFuy)U0RRU0mB5f=P&h|y^lBfmv`U;K6B<7(co}-&lgq=PGy97#D_mvMEK&dUqq-X8Kz-wI{}Z6Oc9Z1Yqm> z{0e$q|2yTrq=T!K2r>sTMxZ*;E@HckY{DhOV%}~7A0LjVX}7`nOOZST9doBBnm#UX zj#tumFku7W;>lH8?i|ho`mVM# z@9&8I`><-U5wQ80a6_>nuRPOXl8-Ou_w9x0?rTl9%?sc6y8GZIvN-wqL*K>c2Vx!= zX3`YwPokOhRfVpYn`IuDUhi!$Pe&BcU{@V8s>8I8n0&xv8- zbE9$r5zj}-)1=3p1bQ4IEANkI&i# z92~Ib5)v+K^XPX|mZ#F_>%Zr$zG}$$e#%GlsPgN0%t7>&+76S(R-wfQzJn^JhK{Cx zFCAQ(?x35`XepG-`_w=-(3$48#FZ@$#~9%| zTfmsSrkvL1JQ*?H^B$8E#t$@WzYBlc`cSS_TgldpV062_>jje1#9J%saD}PPa_}z6 z8QY*zt@QK@i_(${-TQQMn{C*q)TVAU>{G@d%YHsT z6Nz~5XS;zV)XDkw#{q>fwa-MOsQeCd)9>E%Q5uJ|>e?%rIkrV9(3(K@8K# zfH2u#pB+mRcSP?UmjyI+B3L7=y!Qx}TKZT&y$`+}JQ^IG$Hx{mOCS{~^ElW%7rj28-A4^$p@S(1CwerX2%l{K}aT&9~`pzUMh zuZL=Ei2esX-W==En10XWYuiOp1`%%%o;gA-Wc(?NvSd>-=wfj3SJKwGT>RtMa5en> zR$4W+gZ$GEd%A*}&Gz^?;SpDEg_7=@qbU?jBFh631JYy!)}Ngq5|tG3ykc~OAJO|F zk9r_xsyz|=(Nj-s3#~<~b)N8zYq;l?DG6H2=z+R(g00T%*t5~+{ z***VLnDODai6<0IX6w;9i$=GB)~z00r(naC_wn$v5zyeYxMLlb{#j*R`J(l1?zZ{d zA#0Kw-iaXm>aJM(6FTJ!-3e)W@YlT>rr1neE`N-I3o8zY4=tGWxU+ir!2EB(Mm!?G+wO)kR2q^Hf& zg%>xWSLJjYrTP)xidS8*yFNA|Wjz@|S$A%=R1Cdl$z%)3q+k}a6N$W#dVL{W%Ui)2dD&XO z6e-p~JHDmSblGp$3D5J-m;5=Q&bEj@kHzaZT&t;{t?ZF(lh z{qE%oGII+!0KZzTQ;XR>n)$$@KL8JLV|;;Cv!d1~PX&*Lg?1E1!!?_VvR3yM4OZhodp^9}Mq4T<|HD-c5x!!n zK?5o5ztXHxRanPyvK-4N*uRaMebl;4G_wfXhaa{NyEwY!+QXFeX1jv}-@>|-=?*#V zrN;PDC3y8lqtM?^D1VHqysUuz2%Ricp*Wd~ z9lkadF@IQBYA8|H;u&ris*alqzsyB%ycOh+xqSL+so3Ejgtb(y=0j_UnE$IddiJ}X zokEOmTb>Ijj7K+$N~{OmMk9Y0u`h|j4&-_M{BE%DvcP{ZneOoW_5w%r&8q&K2l6+?CC1^f zQK!lOvtRhBOF7f!1CgKQ2eOb;gjJ)z;Qkfc+(7G`G=l^Ja==h?Ru$A(mX6>y@oe@7 zb~1On(0HVj5PoGW`zO?3k}Ji1TlwzUTG{_(5nQZF0-Wj(ex$+rwQ z;P$1pj>hKr(t-mwK&Dbhv28c~X)yX@r1u3+Nbc;6?gA`)V%DL0iC>~X^~4%-*4N&I zEs}Y_CxE*LRHk~Jc|*WdAC*zoI4`Cw0gFI{9ZG$be?a+1xePv3V`1UNx>9(u+1_K9 zypl8~II{K0_^b3`iQvqTYLyH@EBs)3BFgk<~Q`c)>N>#XIK6phJrOEw0{42p}#NwMDr zU3eesY;=R9+d$XD2fj|s5}!=Sz8aIMyig(K=(hDb|M`HMZfsQCgdo|PR^H(-eG)}@ z;n^`xegD~XoYb-q-nEPP6I~CUvtqpK8ILY$QdK~XMDu->(yFY4Qt#EY8V!wa_Ux`- z{oHA0i$*^1?q^a&cbfp**T4QK=kdX0+4T`#U)AIp4=ZMMv?WfBH%Fa;Go}z}G z0=|01|JK|6VoR6-`)z5Q$OKkZ9}T%*;TrAsdGp0By!{yrKTu|9<<)3SG#wfUPdecE zC&gbp-Zj;LTJ0!$jYd}G_|Z%W2q*niMO|MW6!|~lIel?#;&_1Av#A5W z%^IcL|Di&z8pY_6OZtaHaGp->jcQb4)ap6aCrQZ`jd7l)0V9l*7!N#VvG zc>{NM)_^9L@Ka08jQw$UlXrt4X^sW&jx7_;+vd_E*(f-rDf?fA2MPziT>_9kjU(=t z;l$|uGRo?dxD?R@n|6`I*pr)0Dx+UAKkD^WO~?C25}Amt6y}?lkY~ zUQ5!2d)~qIMBGM5NA!?0iGS@__1h7zDNFU^%I$u~)@Ds|Y;ZJJS$P@DV?~@ZQyzbJ z-cc*5MqP>ZfzO5I_%q-4EUlh?jDrslj4?TCpjF?^N3by}caRTWdzYGRP-mX(ms5EL zM7%JU`tww9F+_FLMuYW5_QlQ%uTfHwpE|IETX<3O&!-sr>-nSDVO`N!*yv}<>A>$? zV|_Mf1#Y|QLP{3bL>s~F0xvBXKSWZoY4F*6+N&esBhI5c_?~A0vRbzp)!ZU*ZHpCC zIhXgQn;sP{1f%znW<8F8+BET!`Glx`ZR)a}keDp3dGRBqfU`N%crxllLGa+0qp`9u z>v@PxV}XMBudG!Afy7+(OuZgR2Fcj3LiS@WV_4}<#Fzg~7>A9OEn0pZ%x$p(w?YeG z+vUTV-Wh%0R5!oQO6Hqf`#+*frt`v3sH)cO%;`k?4KGdb4$=L) z)08yAutn;ZnHvuRKfdv4_tv6!Z#5pJaEUnR(%4c&F}^g=694m7aJ$f}BhtO+V6?*j z$8^&5cYFt^T_)eg6SYQQQlaxdt2E%D0mwXl9|$(v_Z(x&VgC?UzPaN05w^)VUOAz4 zWv2IO;I~LxJ1RHo>93}8y#eDN^GWbem6CpO@D$|cms&pKQaA4iu^Bh-SMFQJ<+8W{Cq+n;x}EA*Ad38;?Wo#?uf)|84DhL z@r2@L3;vUn{A`RWN`j#O=Z6O}zOl;pQd}f$ddqV_-Y1xxv_iL)zfs!OYN;AwR9vWD z5zvnMEv)2YVLw{QaDI$uT(J<_TB%=>rhPlGKYpm0G*2syW!ig-B^!$Ic3x1s6LPG~ z(WkKl72G7Q(9P^nDC`|lyxoz5R*#C?1MR=AWbNNJg3dn~hEJ-r7G+IE927CMQr+r|fuv8~ zHW^)lAzI>NQK(#pDqDXl-5&{4La^<-hmr2xcHfVql%{VF$_Hg7iyT@%)&JzT=HjvV z-dN*3=)s$J<0De#p_IRfln&V&X98Xu9u>R2c(ti#e6g_$BxDhbIDvLU7D+SCs^5R& zjQW`}eoMv1>^++=F4}*0iGM*W8n3@(JP#S_G}BuSb@(JY_IfENEhKY4lf<&%>pRqc z+$-zeG2KGsGSvAeM(CZm1_8BEl8EFu|Eg!ZP6pk#Tc79JIe2H=c-oq`X4drG zTl`<3E_?fhom7pCiRG#597}GQH=(~DH{>3^ zJ<2Pk^}N46ei&=+VaGaM>x^qWeYVY-R4w+F{q#@TyGeV<_SW=Z;V+HpzPFe9;*T#Z zaQxW@hc(P6KEXe@w73crC5GY9?BAD9%NT1*m#zHU4oPmlH`wU?VBRk>XFSe*()H{> z+dxaOdl(w0GaBypSyB3sQ?NQ^H6`l#=I7aU&(X-se);fInZaM4&yH*+#RHq^4#2PN z{T}Yx+82AEGQ8>^mu(>nWc-*7PIm3(7Hr*ApE)?5GXMXk}An_;FuE8-KEHKlsky`A4M z1RDsrQe(Avu%Aib%f9DsamT{Tz7f9Qy0 z`Rp3dv-{_rhcVwiKEd^eW#_rrzc|;fPML~`)J$fuXcxk3^=gb5RZPTXEHXBl4V*|; zeobc97_qtG6xe5jG`M%9IX<1jAJ(@9?6AZBn9{vxTaN}GkWT%3B0ZUHh2lgUo)#tT zHtv|}d>0!&Yx$P3mN+KZvo-U^FRJiU*)49A&bwU2i!jqP1mFuZC7iPWO2>M7(ra(e zrN=?&p9|im{}n0NZfi7km3bkd>+Z)ajxmjH??R!t|Ae5b?TzghR^IoDJ8WS_dcLE( zHvSa>8lLl+yTY&xKR3bcrH`UMQ^7@eZLe$)D`s@x| z3nRD`g533gG?CDOOT3_Id;5~)!;HwWJkMerrF1Jt-0%O4QHi;KepI3Nhssnd^qbz8 zTPHL1Eb9{Hmf8DU;(ZCbd&G{OXg6hzi|NgqrgFy&ku-~%2rrsF7Hn2eY5sw%<355p zIPdz4IK5x2JeB;ik1(g;pWkFG$9cy2BlwOyRPW1gDdV;36+2IFZG(Hn$*Z3RC22KD z!ZB<9SMp~?QcLzWHh=LaJ3atv z;z#*pGK+E#Hrk^^4s)f#T2WOqRWx43#y%<+H#T1PYO5-t7Endz0&PjHxjFBPn_~3} zsn@nVW7Qt{+D&1h{!{O7Igu}>p&eQvy(CR{h7S4Ps>5}Gfq?yy_c z#tph@JIie4_#Y2b2VDSrr)=X%$l|J+*)5_ z82_Gfr^rsB{fi8e$eJcga~*-d)w>Vla448_ij!$bqnsw#ArW_M8=(m-*TU%3D8E3R*MIZDzxi zti2w8m_NvEdMmiea6x~b!z}ioIW-!u#diFF^(y?b$mCTao?4vz;#NG3MhR6j?Rz(9 zb>3qAEr>S#Y>ALo`S$JJ?+-y2AE!NQKKdA>Tl=m^=WC41`*xO&+nr(2|FVIvpYu4( zRq#w#xMsn${l0}!OHOXi&!nE_6z2#}H(8m~cwNk)1X(ZP?Vngr%-JIg>K)BJ7MJh; zTO5=8{d)RjGy@>L;eAPof76uFTfg)|=}n-@-v@#vS1FsiWxC?S7AJ2cE%tNmZ_nLu z)rw7qa;SQu;*<;bu$DvSewTQL>W{)`-qqr)RmFy0%c`W!!dt}F&phFYQbVN>1OP`@ zJfGzgXKuim#jmMb@fImCpI07BAlpg{TaW$g%#iKhq8%S?#diNX_*)x!wAY(g-UnTL zH1MqYh%@sTn0ftgrhWQMg`EL(l5pGp+IqhB{*Uj-r-!*{GvoD!gkOEBNLM;n{@tJOva^0Cd`>51gmOfaF^O881Y&$KgVQ1o!O zj?xwJQ_GZeVd%yA&d=ZOX(u23&SEo8ij69-Zcw*+e6Qi=`_)sU9&JgyBZ?tnVBeVB z|4#DppKBPu(QY?EZ4Sqmu)>Wl(d{2@;|^9U7CZTb@1N}buddEJp3V3B|E&&1wN*u1 ztF2X4)E*&eYgbWKwW(bsl-MGwR#i#uO>Jtg*rSvn2(`tiy%HlKM2uhF-_L))$Mx6! zxbsKu>zw;K=Q^+Fb?&m3A)WA{W_2f!`!~p#rrN3R!8dZ3`*2x}czvjqQMo#W3HISU zA3VPN+bF;@%+JRtCCLZ(6Wmd_n%1eECNF7HYw1NwRg<>K36-`Nm34Z`JxdNUJH216 zu|#VlA#k%#^(H=%nFW@_$T=#LDi(cgO;K`>l)KnSOmpx545q(3)ZY{+`BUBQoxCUY zdiOJMY8EPO3uSd4u+zF9({l91y6UWEnDz@O)D3%2rb@0{{AYuXXYWQ%H9A94I`vto z=gabbJOYG|q?+Ji{0_Qnf&yAEY52?K89_ zjkjEYY`Ij8b2Ub&^(6fAqQIApWlo947GW2|7DE`o=z#5U48z?&ELl(i)__;Cx_o`; zzELqyh>t`1V+qio=8qDDN8cNneibWEdoSNJ_*+FvyL65K2Ac;*2mJJGYVoz*G|rE_ zHlPVf5qY)FM$~dD$f#dxxwpS5VGfHw*uuueZYDAL5zQ7Z^vKTd?Pca7At-at%%*C^ zAA}1%6^vC14x9hd9#E>3^vTDR2|IQqu2%%)JHALCo^MUI1E$<>;sdRoB(M&LZr!#5!A0tA`4bw##la z&3|zhh5~+gK93`XP1qxZr&7`-oV=xd4WaP<$`p3A=(cgXR!}_#;9!GHgd7>;=1OLV z2d2mZh@NAs3HII@K5W%|)z)FzR*zqndSly#v6}&(UPeD&^i^$LeiQc~#MkzKP^fxn9C;S@*I0>3nk)GTXVO%8EPN#S-&ufWc4K{OO2!3o^UR2{RkCUb~n@ z`gs;P6;cq@SEk@)>e(f?RBf$FvM<;)%jCJN>T~ScDafA5>4p0T?rNMR4}RK;8V3(% zL#8;QIP(dW=XX52b0*0da!^?{RzS5{?|c>1#dG?5)|8JxR8NB2V58OSVA>>q6z7G8 zco%j>)ucMG-=HHi>Hy>U@*go_r-#e9A|%}@69@b1I(BR}MIYFr#s=y2w006u2GYKT z-EDE$x99htWpI!=g!Fhq&GAec=(%wmK?1c5n2F!=vcD-h^Rms;&i+M*^&}rAMN%?V zd4BcjT#B%9L(r^;l4q}j^)9L%y!{t4IhAPUwz>{uD-G~1I#Lb-d2kPDQqF5Uo9cHv zy<1>t;bMb1U3g1X)HJRz%J!gdD}D@(a8@>6Iv2Z0Y^Zd-8f(XYIsQI z%G#!(n6!KVTa!lRQ}qCofd>4NW}SVLC-mbNMTA<^itLAL0Fy0QB>vW+B2?Pu5QB}G zJv_>X*((HFGh$*2qS%wRBJchcW&fSDDmtHlmG_t)g2@borlcSI13zAZrUl9IyC!bA zY8V~*^>XZfKfL{Ew5>Rc=QhFEQ}u%FirK&j{E6+s(;a8HWA)Rr7GG)f{z&tGCBN|v zM|PDKD|rZLkxNRw^(kgf2(E$j(-8uoO3B*gH-q=TRCn8CJn-h)M3H>Zb@pC_$^Zt7 zik4G8y(*2gh77;IU%_Zn0(kBlehQi1Sp|PL**9_x)CBeLJjwmdA^LsL^TWhaIKIim z?WMCNtvt60da~%4Zw}szeRL(&XY^Cw{=+FQ&jNZhi)iYAqIYTdW$#awxp;&;*)MTObEn4d}x z>UsE2(a2y@J)Wqv)XaE~3T`^&R5urvJ5MrF{U&im4RC-9IUSNH&4tvi#U08Wvs#V{ zO>gzvQMXoFYUYzS*!R4XH?q~6C%%}U$6BtMjCZK5v8wlrCY{{Dqi{2oe{?>pn>;Kq ziVd@4uiNhJb)5NnF>S;dL+A7W5rkhA`?80V6{wb&t!r6Oo()7&uj+F0vJq|BO*-ifg+EINm_tkggl)XH!+EOh zp=Dc@ox8sWCgn7qa;4|7`*bYhDwFj5vZ+j=JVDV()H4WZ@Ny~WI^hK4qq^A{+IoY~aR zWyT4Lz=d*s^7X1JU+4O41}mFU$pONsSou@VM$3Y`s;OCQ(r^82ZEZdOHrK$#=G|+U z@dffsH#tXa{U0Z!Zugi^YAO9!*)5-?sB;3_DDBN!KI&V0HjYvg6_2 zMTpGsEY)wvEJ3-6a%MRc4h~9xyqys{_^mLHQMt zd;7iC@lAFv{nBoFccp3qq%Zkh=V=NQWl$Wf$1l`+Ti(If%)=7IJwaF2+4CQH+g%Y& z;7*v>Nw9CW4{F*f=8)sV_#GaMu%{mHW9zMln*G|`#^*w>dfo~jHQA5mUS`tozZqUN zB=ujizRa$g-Xo-`#-1=%&32%dTIB4=P#^Nd*aANR&{hKLi&O{r6n>c4PZL>bCRj3Ydd9(RRK(;UdPkX0dgI3D@!CEy; zavoXC$3CcyK{YNXE8fcDu09Zs&6n+hrYa#90S)u51Bg3)@i)1$W=*MnxpA>xRY@NCKU2eZ6-^}kUKkqj-Uj}{r zH=t)ZVXEzTy=W_OhWuM>MD!$V#Ns%4D%D^v0*X*@08dXKt3948ox>T+|M>+$8~%W1 zXJTdvD0`6*`{*-lTL${O?}#E8=SWv{+=jD z0=JmDs{Dhrig1g<=nse33mPvyv3c2P6t|AOlTjj;lN~m!-=u67aGA-qut~0 zxfa%+lWM@=RJS?yu&88@jkw=D1?MZsNMc>>0sD;8nhUMcfN?__jFf@))g&<}uGVNMJh- zO^+{-Pv21W33dP!rAxMDIRDc7uGu^h=w8d_;j7Z9&g;;a?sp?C_r@>t*30kXM$Iex zYxEWJKPFzo>cs>Mmeu~fHtJh-Bit0<-ncowWp4!cW9tKRq!r-pjrWDt2&wGY`&=LN z=7-+d#J;3$kcKEkD12^nX(`Eam?xBATGPST70u)q|IoS}-RcGk6HgrF?^Ng37A^d~ z=QZOkRVn%9^$65l+ev6btJ{p!A-Q=+}@k@s}0)E+QR zKPw4gACEx3O|N+h4l}p+s9|n&@av*hA@6WR140)+@Zk5r00))91qL&z7|Y{L|f4 zS9h+wa}8>!H$RSqN1ootR*mz)-2Hy+2*8nU*m2r|jH<)$Na?`|`r8b^mk~`bIbQIq z%#Om=bEPA(??*L9hQkWk96VlTm;9%K3TUxy8OrRx1vtQs1;8ri!MZZwd-=9Zh?95o zRgV$>71t*0z5R6u59RfMxBE_6QbDx`nolr3d|jP%Y8jDR_+GRsis`tWX}xgwyL`%{h-lsnS!=(~^w^_VK}{ z4IbeEwt&d$f--#k7xhg!XB}t;T?Npa<~`{GZgU)z&=S>hCv~V1r9yL~kan>;OO#il>$UBKT&UP7Di@i!m_ry`>}#*WD(qM zN1b#5E69MQh;(m}DYd^=Q%25!FLJGIF4M0mSaY{OXFJ3uxXgmdLDbbEvkieTV;4(J z;F-}+W*r4FcG*uKv{lF7&WqFKeSg=7Q^`11sq9Z*`}rcMYN0dCULX_Zrri-ldvj3Z z*?x}mq&Vo9NnY3Pt}FVRD>hXQYA5Zxzy5lWQ_gMq3~AS&6wSyfzyEnp6)X7ANdnR~ zsalh|nZyRv14Wv6K3aJ|0mJf zN!||j+=H|ZZKMH<;=pbu0M$gNub_i1kn+2c$2*Pl z#(*4$^3UwOTVA2J_1BX?;K_E{$uiC}^|(qD7aA+x;3u ze2$f{XGb3Z`%o!#+120DV%AcVbh{Z}W>P0|n0OpDP*F@Rbl+4E zpg0={9jGc=e1vu!J(N{MZ>Kj!AM#&c?h-%g6@f|7SEVRo-#cp`)bxlw3{=i0IW-5Ft9U!k#9v}v)~^|f^7@N6n%_1=`!4}7KdRRX%| zY)b3qQw+vKQN?@w`+{5`g6CZ3#BlKj+;G7)s64Lu-)Qudx-(5G|IR?!DL&=IpRoYL zkr0a~lBUTROYRJ8R346o}Q#47?g!jepX}aBxdCjkt61S!y8Pj`0CPE2u@1A<0Hg6c* z9uEbW(&^H~^zaqG`vRMROlK&I=>94YM$bAlUNWor@WPz62^Ye^wo1OeKv+Q&;f+o1 z23vPoT&~5m{gHDTFX5O@xWi{e*$|mR;?EbmB8ZjKhwddY&9>vNGaW!VX|K84zYYw_ zOp_tAjZ!WaX1#`PUav{ECr5h}N}_p?WX7>TWYba$7ss#~W5z(#vd%?vEH-lnDL!xO zCQM)(IEs+)N>L)K?=-niDeZ~`L*xW0s?}%=N9q>&-51eOpQOx-lx)uVmMtOf9lJvu z`)`H!yLob3KG>vSU;4{O7jm};-Jb=Vc7)xB2`|CS*)}RWt`w;6N0KqAiBN^zYrH$? zVMS{$4EnD>9p3C_V-!Q{_1QyF1^Q^6up~U@{#+d(N)+D>&DmAiK8t4zs9P(a zNLo^>I=VAh{(IQ18=L2}@#UuBiw?KMb&)qOqkr-;(!F^U{Yh9)ZPE4cKEM#K{wXHY zMo_LHHU85;tT*p_e6E#Z?_c?CZ$7bulHxzkBc8lkl=Z~V!^o`07s%T`Z|90Q{94i; zoh?2!uC!8do?s7xX#AX}E@k$Gt=1j|C5#l|a9^csZeD9ne#e~(Ugx1)EtA~->-C(m zjmHwl_f9n_DPGNk0a1>3KXRLK=FMfagc+!2JPR><5W7e>r_3*y-ibOCw5DAJyqRx% z#zlipN15xse0L&PWwYt}`Owrtc}Zc{{x@0Ep=V}ruS3eKq|buWXHGDOaHs@cNwiav ztxF&9^E_>EZq3Vsv^^h*vdEsIT?jy~F*Eivz+bduQUn7AdpQ(`%A0(+JEa5YU;7AI z^q(GQNS%ay`%R7*kWo7050Wzx`fKIjdzbQuq{oInABYUDRrCMh01ImdW)694|_G;V&ZIy zqOMg{4b)_8xIdjnnpQ?FF-;R-9@8}rKIhp{{S zA$vom86NGQ(r;^1T!#mNt{*8vcgZwmy|3}(|5Pdj+c=gi&QxYDo4XVLGIX!`PbIWT z`kS0pLFYjW0Dc0<;WC-hHQ!6E{mgVS-+RaPntZA$V~=(~iPEo$&i5v|o)~4%ApH(` z-t(GU#4Kf&{>ojw$ku%AZQQ$=4ULN11>ALtnD`;y?Ce;RPNmAB&aL{UwBPf5>|F?c_-~Wmla5~E9DuA8hn2gr%koWUcW?%ki0_$i+8Vh~ zJSBBJI5YcWqt>hTJX*sdqcwgO@9CM@`g!e`GQJw{Qq2v#>OiASGd+KZW-}Kc()U`- zG<>5QZgomH@n_q8>zTdn?Eh-vtRWS86>cTiM4R-ss%Bto+kC)huM9^;+!l^H*hn*j zCZnu`lX4x+Ba{dq1?4@q1RnhC&yyT*4Jz&eB$hj?_W|=ggBb?VaE7Q!AMDftgZQ^+ z>Tf!JJXc_dlrW_0GfT;8QrJ?zdpkEb*BNI%7TYsAlCV~K`#UdxxQTQ6E$rAfHB*e< zGEx(d&7~ITa+)?C=+X^wR{N4VtIZKtYLNcY1^)cyrHvntrhI)>NHNqOvP-iQe% zzDp9g*Z_$8^r)0kxRKL||4~HqgVis*O?$12gW3z6OV}KNx0l8i{n;aQp_^!%gFy|*k zg!*xxszv&G?6m}+!KQ?|ud`hiev_{_>F%`UY@<;=M7GBKehUVwjyxaiSMa9 zU00fv;dDKae^qcwpSr5i{>l+mBg4T*h;TrE^N$3oa-tJO%u={P>bu*)ErGdB?*5a> zcd&w+n$4K2a7~!djFGH<$|4+kq$28TwQBy)hdh;C@_$%p5}UG{Ag zi`3jA&Vy-E%BGmUCNn!bn+kB-92>@7!^1V5N~UN-+}-X!$KY3`Iz-rNrz&`uj&Znl(Q*$6)h;gRu}U!^8HIsJ;yFPP_a8 z;pV9Pqqk=^lc?U4w?~Bu?t!#)xjqYN557O9RvwK;#W;i6ebPf-$g5Dge6!fdT{UDC zc3b&P778TeuhFTv-?VreF7`z-JjLF$Xd<(@f|U8kPeOKA>wZx;)2*Ew4>d|`t(;|% zB?^Q-Cp8BapcdCIPe08kWehe7rtjj)Mhv?9jhvnr0SQaDf{Sr_@;psYnm%)gM&eZF<@TVC*`Xf2D6r6?6Wj)0E|301TC6ybe6JDk^5XKMm< zWf+x2OSyfRRE=8PmkY80bE(hqFY;>&zZr0?S90w(5XnDN>SSLK9>+dNevmjMxLKUI zc?Vr;0K*B@>Z>YP^3-zr>b|$DF5z~do*tz{bNdUf=B*6dt%s&lDGtVnQ-)dbCSAKz zE0dyuubR#F7XuDGm25oklU7=;0xt|)$%)q<6AJ0ObfDONvfz?9Au1|-PitEYD3ZBM! zvwd-H*|1!jGuxubNWs(V+!-;o68(epJq5lErMfo}vxOBQwdo-pbpn`j6|Bw`}_FCnDecUweG> zf1Iymz|MpqMNrQTNc;!JF~2W2YyqziVWe-)V<$&lo>T0TvRD2?Y8bdCo9&^UWeY(~ ze($$nW;c$K{|yU@;0$M&JWxwCORQR`Rz!PO&St}tTRwuQux=jNmL&f~=U@cc2{#JRYZ=^B!yarJ z&gmLCf);P@6#aZOysOSax_VRA#a}?w1tlxr6~d*i_`zKFZ?j~Xc`gsG&^c~wB+{)j z%6*e6#@=kAYLUH~9-Cb`k-q*2U#5hO#P~d4hZ#DU=qHrP|M1yN#q+DOj+r0Vh1q^> z+BeWrle$u`(Tw=`ro2A<0@v=A$66Wag1loFXToF3<8*nxF^#37C|7h`*(939sgZMR ztFWbl$#Na2@7Hx~DKo9n8P&JB5#o;q5cR3ZCBVmm=7evE)8Pe6xx!G8;tR^{K+YKT3*Q9$fW9XOEXs((L5CSY%IL_T7<2w|C4FqaXJ$w zhE=^+<7Y&e2e8)B_r#7dcixsSs~*1XNimFUcP5| z)aLYBnjrqu4g1n%(EO>1Q@|R|g=AE1pZO~-19SEUrEDewoL+EfW(Xn7>c7kRuKnx6 z;jZTaOSwgfH$e?(UsuKyDfeh6sZcXD&;L>6*|@<*cWU_)#F|4$zTDen4mmS@XNt6M zE`Q{qrRDh{gu91b0XzqxgqE!rH0z*a#G2X*i7 zFux?H;7mQfm6-{(r(bsp#^W2?!wn6#FPA&AIzNc2Fise|gewKN*y1 zE%5QLb_T!n)szrhnWRE?jBCEDgE@M@ZCZ4-da!cX&Wz{&5+}n!#h{zSt5(Twd8>55 zphsWP>qZpDpvnaOu|VEK_J zU`>bquz@OQO1`Vyd|`FKcO9r2H!)w%Vhb1h_WsUZhig$*PaX}#{9?uew$`AfQsc9`i^e9R4_B?+3k+AO?L_=ho!}rQTNqh7Lj=SjeC)MD)rIS!G2k>TWA6V){ z^RZvEgYCDOL6(0;j9%Bw$Ts5Ucke5oOl~0>CjTy|TKn&NL={4vDFu?NPzB&%bhp~M zWTGwQ-HIkrZA$KSI@)c2pmg{Q*(FOHMl}i9MoxUHEN`h+1D zN;f&yib2aglmpC0h0#rF=iuw$BIh>Sa~+gwX4;^bn8h-bq$-bLgDL~NJFwhbpqpD- zr7o%t%>m4PTrSnGSo{F`|M-OM;@@P9y$R4?4BzSgF(T#5loPPb8T;SgfQnWAp5pS* zZy3UY2*A0*AcXJoOwURuW-Hut%gyI4r)%VD!JhM-zUupb`&g*cHdNjveCwwzFU!|~ zj{LMiq}MA{DqxM;mfUWudHc-2Wx{?tDON9=-eWL3(99FF&zf6neho|NJp#Dg9_V z@!rig8e4=xZO-2DMd3LeW%~?5;|r581&6=6m$cYhtbcM1gT}gO2maa14QQr8QLQd`FY}MjfH#|@%dim ztPfZRLh_nZPw%9@t9;URKvb7pI!`(VC}@i~9O)qc?~O3Mn9=E%H8keeoQ9LUC_`@} zLvP;n4k0Xk4d#~fL?j*E@N*SP9-2bI%{5abj^D35>W1%m59^n|@Jo%5Oly2#0 z!Fz2tO&2lXOL+n1iHrd1BMTA4+{(#pmu0+DW*TE=+MK)|MZfe4Ao7qYYX+q$)wdo; zzpF53!7zJL5(+b5EdKz${KCwded?_Ry7W)Y;u35~^(+!Brnb!9C0W(Q5$(b{(o@!L zLcTxjoykV(?xm+1e3j)NyIvzHRc~Ldq^0BIv99#Q<)M|q0Lm)3VWRbk&Zf!A8aX>4 zzTBT zcuQqn-J$N8M^daph=tPr_m6d^61>RuL_(D2K@Zm|6sWsqnm@eCF=({HY@I@u4|Dil zx|l9uUpF9mFzgiMpx(x@({^^eMV=rjU5)f=eD>*U-s4wSrt4@Fv=HUqJADP4rG}M@ zEM|CW;B5i);%v(>F>o%3+qt=MOA+Uom+r!09It(l{#No!BI4fD9PyQoSw-^-ApW1Cg@=aNBFOwW3csrzt z62x>y|8Lzo-lbDUN%Ni<0zr-2ja$-c^F7cmG|&Y+9<*DQmCOPcF66ctUqi9Q3kWbLC;fC?@}G;W0J2?gE&6 zITU$cr}fv}Ek*l&2b0mMo9w=NTUibqCiIdsz8Q&KpzMi%0>sq;lP2&~RsQ)8RgVCX zA@4T9INc{hbS#FsDsYpdvfbALPmey9yHZc;w*&-Zc_X{7k zK`@6O%U4=bGFLULulypK4@YL--1Odg&jUZ1b}G+9ASFLGC(`5gfTM4fU7xUcQ$NBp z(p$fI8a6~oJ)N~64O{%i*UbZs7#HqH1>gK;{@~bEYVKyC!zj|0m;0!2rq6LP{$XTT z^yX7+2HJ@md}QSLI*un3&Ejz+A9fH`cU_D6%!H^~P+YM|A=rBQt@n$4*9A+6pu&@r zoz3F%xqHx>4!vHd%NhIb28y2r;wvvMVINF3x4YzOJEh(W2M3V8w^%4V0Z?I#Q*{y7Z*1-y8`q`rz5Hk_mv45sjxrRP<)ThQ@0YcCu@$$TmaNhNsPJc z{Egq0i%Fzkqbn$i?$^DP@dzIC=_g%^*9a9>-itcoVNRFYK@+C4sE|9jaNTe9CGM4=DK5_*_8%qu9~gfx1kRJ;G#5gy@W|SbTGf zcgtwk?{i#$1;$<@*L8H!h^28`=C1s!*pb=`V8O{G1{VnmDfMY{XD-D zVnpK!Og>euf8Gu+exsUC<9NqKYUhY9aThD-Gswc>;{%Q2dAG_vrfrsK=0bq?d&=*do1kC$#l!vjdUE>vHPO-05~IPa^a8)`W%9P& zTP%nm72ngk!T9QlRHzL)d#-ur&kae*#kXn_Mil|)&$E|VR5P_Ka6S`MMh_x@sTue=Pm*wxErlbhtQUW&6A zSmmDFrEXhZqzjFy_%HY7$dMfUPpVeg_SqM`ih5hiJfP9l`al4$3lnmSJ^{dF{vJ9j zt)S5C_u<2b-B5Pt2hK04*(&qfRdJ1?=W6eN+T*8dTc$C%Osc5emNND^W%8+Iev}>Z zJ~yK;@|l{JbL+BE$DLlur42{-kzn{ zn8yY%_dJX4qy8pY`y+;dE;W4`(aU~<_wB;F$r9&Ds;}Y*XdZ5(lrUKQDYS~SEzzuk#%Lwgje@HO``sKV+po`5!4Xm^8PbF zI(pU(EiBdOJlm+`JCv;w;TX`|xRIag_-^2c^IUAJ#c06oT^`NVEqT*1T)F>g${;*m5ca>j3;LhoeWi|DSGHRVXX|g2 zGMepaGhdp?I%My|ybb){@OpdE*#cW6WWGJ_J%4^tx_nJY8vJIT`t8*V8YFBsxN~Rj?3puX&g|K<)-G$U=Xv(|q^+q;MZrWtKtMpHs`BCu0Rf@e z?Ry;=@b(F3)sH720ATGD6}43r6*;usT;AC^S`!dRMn~(;yUf0&mHXsk^6S1_R#^-+ z-Ir2b9pgG?rs7VZYFzH-bEdlfsi(iXJ)N0!CdI2qopdu#h+IHtfRJ)hwy!?VnLvL} z%6ADXRdp2T#vlEX^_Y>jjXMeZ<62Ux(_q?J`>ixs3gZpk9~P7E(ob1QR%L`Q9GZ${ zev8daPL$DmKt8CzeJr1!=dW3BB0wb28Lm9h4tvnLAVE5_edF$__d6f(DC4=KzHA`d zfn7-aZ$HL2V?2|=>xdX^DtH9%r2IfAAD}3!DEbL%Ooh=@V#4o=ltDY`%{k;+q zK6eBiv4mA|YVmI|)%W?m_S#J`A2F!c(!LoKa}hT>@H^N%Tz?$9+z>l%=G`@RC%D9+ zPW7)KXY=|Q#>+EwZe5$oS3;(U%Y}+6IxYo(d zcL4e)BF&**;n^tnt$Lb0I9Bo}(YlHCMF2w#muMEe9keHHfFyiY z@HGBL9)0(%|FM z`wsYTX?DDsKBTC2>dIum!9}US6_WR`?72Xrf5rCJU3JLsmfy3#abpyFg}_~V1fV{v9;$Rak6l6EbT_JvnPJ)#~Zj$rQFx zbfxYb$=MmaEBb z2@Gv7YrpHPI~~*4&W?&mC8h<_i{xhp^0Rc#9TOb zK~v!Kt*g;R7(bTmp& z9&$8Ie$WiL!$JJqZ~sA)P29e(v)A(4e#B1R+ft(s3w!_EeOp znC~0T;wTw0>Ei;_noRBY+9LsD=Y5YC&JIakcB*^%c$-qjW>M89Ua92DI z^YFxYKhtUb>|~}%f@Gd(wDrrddM&kIsD<`5Fe)<6-2ci9gu_A=rqNrr zC45pD>1INgn@)^~rC2yzzfpy@{n^mofDmb3*m8I_L(w8AndGzGyp5FE+eQvwVWG_P z1Plr?n*G3oG+zbcY?dqL5kexw9B<*e_$rxTMlW$=wD9&20OZ0x!8tBvHd5Qs0%RA* z>B~Ix91!rBDdq7`Yoo|4o#2n>>upxr$GJakwymX1ADk6&ZBLB5`#d%XR)VDt`0+=Y zVQJKft{A!eIQ+6eV9C$V7@z>)k394z?}vSn2WY64z^LTm450k*o;ai%lrs<<)Zl^7 z9X=r0u_xeF#kEYC}lxMt3X7+we}=-J%-{ti?0p_(6nk5XHp{*1=FO-iaG_>;n_Bo&@v>lMs(AMN<}^ zRC+p4ar4brcyKlA60_(w;(8(ygK!foP;TMzT+&M>Rt~M6UifKFV-P$Fqw++5K`V7V z!%cH$fkLvg1R>XKQ}130PqAHW_zEqypu}U|C_%K}D(!rH7cd9NutlIy7qk$n76Rp0 zY`nBu8$}Fh2^K=N;gHDkYnNL3Jp^+$qtE4g21-(tVC!)DW<-d>D9Rx5Oe8=*nfiuc z*NG?+(W*Gy>5Hsl24WP9Bf$IaFVz{>u&j1Nx zibm*8yoDsaeQU2PQvf|Eevt^Ubbu8bq7ToRzX*KD@T=lkS~PcSeG-?`^faYWBHLI* zj5Y{p>U1q^+lxU{!i`x-)8UWQiPm&|?(l6tm1J=ab$QNt24x{$uQ3Uuz}6XU^h^B_c^e zkk>N?F9iE@bF{fW(?3v9O*~t0*cz9swB$$%aJ%)%cM^Dr&pptpepqWa+MAyTD7h`g zUY}SpXyHpGSuXaEiV%=o4%Aa5FABq>zrbbLNw$|eFsclsKGeWJMH+ypqRr{rJND_$OW6)r@3d;>}bTsRXS0|dh^eAtKdwvUvP ziOd*+$hq71A8a(l<1iw#MK6C-uzd`zd0N>PJgfc1w6yI9{LbMEX$t2LhiR7@D*0ze zxvMuNWZV1%?x$f52T zi$M@`0p*Mt&y2nNsX_L@@<3lv^b;v#kM503|eI_%b(Q8T>BKLZQ`fk z1fG#5bG(O}@yf%sF%8(wj>Dd|!fu^<%0?FbR(I!<*JT$VCh9rz{k}4?E^Zce=NAcY zMtn~Ly3Dnl`D2%ScBN&P))K=RsF4U5bhJ+dEQrzO$+S=gyB^njKLNf0#7hr`b+u+K zRkw;ob6Qj)J_UF{EVX}6e7m#2ONeAlOfdRA_5v*XCqv^?0GO0ExzBfuAOwh|hGy}QAH|S>q z&TejXxlk`--3QL<8$*&70li?HMLo%~9-qS? zb|~dNw4u;cctEOyi?SLp$r&r>O3a`#_yJ%}q8?c^X4CEg>~3kJM<2O9Fo>Y1091^U zaAnz+@(4ucJ1}E-S%twNsc3Q`vhaW+Jr%YGICV*S(2HyJ8a_Z>tEw>1JXkPPoT>3x zG23><9q!tEUo-3qeOK~6QyU1m^W3nL8n;hvin0R(YArZ+NhR9hXN+lt3i`$q2}54h z(bnqWPqJ%y)wZjQu;QXi;d`9DCX0Y!zAG`Z)ci00gFOT#@P zg_MdzZlQT1A%jBvOQ9^r%k;-Sl$BaiOzJl&%>QaycV*^z=n$r4Dl3LAjMT`=G}2ZG zdt2WK&ZUg6h_m6%ChA5aq;({q@Kuso6ch|}^1Qwa2@>W3ZfG}qfk+W~cACII^VHfa zXd^YRInOcC8fli+n4Nzi#su4zJ=pj)`G#(25QsK}<2@sIbcX;L@vr|RlAg4twW3u6 zC5WHATYDiiHBUr^g3qx!Dhmtwj4&2Jz9AS0VnT%BL(^%r?r4<_a51WU;X^*hn4(x?Ww7kr5*YQ zg?E@VHR<*V1MzewT`vnb+*9CPX7e0CFGOM6oTmuTApYr^0d!4-4Kv3>NX-Z)6j-rS ztyrEYBEaDtWIE*91xw%N>?&t#tSkxab6^f4iw&WFl6xs^UxJ3zSaa~kV4s;(X*=@~}~T(9&kR5As| zY2Dp@rr~=m@xb)wGYZ~gC3C~@+3APk81QKFAcw6f@58)FJ%DjGai)QnUXi|Dtmjkp zpZsqU`|||VY>>G^v7214j^+PmVL^uT+~4BpQ-v#9Gd(8cFq0ml>&hhE&h#^^ z;aN+j*fBmk;YpC;AO7Qr6LeBe*nCl<+NE1kCWUBbP$eXAIkLbzzCg#{KcsnLa9WGe zs1(_P43vCp{^W8?H=Ob#BXk&$0S>x&BR3mct#tXukP?|zJ|Nh#;t-6xtXSM3D`Pup z$$-o0mSc1!ZK{)~#PWhqP~U=rk3D3KID&L=R6@zq|8BtVP^7tygNGz%PXk6UK&eK> z4oXxknl;&=r-84OP{!HUnvA?S7P<+VxZ;_5J(3U>449E-dmOs>4QE^m46^oi&2%c` zUk)!2Oe3tej%Qs7ZKai=bS6e%tN(RdRm;sdTX%=|lWIHU_01y;+Ody0} z6lIHSRAMd(qg*P*it-?q)X>VmWSfbr9|g%&$%ux;qQI~Hv!`itU^l=96qOLsCEbCY z>rDri*!By3*wC=y{L^Xq5%LyX9TBH)fTt=tygyf=j5~nLn0M*2qOsgcK!zP4 z_p+s_!+qgsz1Nj=%4Pxq2q|oZHOmnW%Hi{`JdD@WDYO^eXN|P2S4(ZtxDR95T-FTR zP~qq|#GFfy$u@7=E?Uq$K7Tw~#6g704%fu*SI?LVln*|8ff!aFhY-GFBrbcvKPYb{ z&jhsoL=RBjf)NAs$m)yof{A8<$E8G3K&L%2VErdorYS?Sup0T{X0gqg&K3(S-?o?Y zsA1CP5a~%cBRr5^;ZCpx8zrlJx}rH(JePh&1%qbjDaSvNqK&~7lxJY6#>MPCtPs{} z;=U4Wk7_@J{>!>6kq4pE@p2jM$rB}JEn+8Q>X|c4lU*V{rNXeqet0XjYJt*Jwyz^ zYCIu4V<^c`1ApPn2LsO5i%m_WqfM~^uaynmKr$`F`Y8S%#DYjDsb=^I@_tw`&QR`7 zUfT=p%D~{E^>A%wKI4YO5U5b`)u8kSI{+J)l_8xaAZU@e9sW+(ASTd_j?hXn-?iIGCAm__zpjgan5eL$UfMd@DsRhu!8wIAwE$1o3|H=wc z>=DT%JR|5kdI})|BK8#xEl8VL)k_1zEmj!dY^1$43W#R&qF@3`^c`(YB0=K7R#a}Z zvOMzq$wcM{dK9pkv8dDnUcI#V0e*T@#uigiuR4P~fnHMY7=(``uzHDJH43Ve$YiGX z#%=JCEgwvp2Wux?`;hEK(72M?-uS|$yk?lQ>88w!(BPt^Kjw z7aZ_K(XtipPBL}sD3U8+La`ruWlxw!4NMCTyILYdQU!sjmOx~!PmcMIm?3odVkak= z>pP%dIF5g@hQ2^Yhe4fQA5yk3VXXVVJRA48xxX_!w&iw%!FqEw%(g4P;Vm=FJ2n|L zFa(ExnT6bOEoMa@<-Tg>L$fo84?VG)bsHME&SIBUu2f7_ZfnT~*w)}Gg17GBG3MG% ze>p?|Py*)7={JrqcpL9{JP)j;&TE&&YSpsu@Wh5j7qU=}qD6-k=3C*`)8fhLcF$O- z#K*|tiW>pFwaj}2GNS^ngzALMwc33yLrsNP`FHr26I~(J>7>l)nk>@>gy$tG4|ga% zdEoMaA@+&(pnF!jxU;Yc#SvOXdq+hOl`%xlr^X7LYX*!{>wJNl)H*+I(o^C8AJ8+GUC;p)q11XyTM7#6EZl@yH$XPj=U@@En zVK5Ayr~%uyW3mB0Yz#8j>>F}UxU^~ZolF=BzPh>0(bFJ*QGx#qz!&b0I!sl}{iIn~ zFjX6eLLKToDbf0X`${)j!f#F0n=nM_fvjH)v0Spl)>4m4@n?%CH;biFdl3Lyg|p0h zrDz?Gt%nE}Lc`$~&&wdaFkK&xil#7}G~CEq`jWM%RKG_*i<0&;&V)4kdSaK*hwyRx zRhHQ{oI{3yEfRi8Ji#AKCK^^`kQi^}ooLxK^thr{ORiz82G<%lc@O6#NewZiZ@nwj z51L{A3K6)>Nv8`F1W!E;3Y@x9dCYA(AJ1JE=`ndhMfvN zs(|nLF1=%7JQ1ojMSo#Y4fDVDVGFlRv`aq?iA6)ss0_gPq%Lm6g%)5N!p7A85P99D z$g4f5gO%JE?9jH$X{Dn}cjKMfFf3b+d2tIAGeXqNeK$3k517ti$faFJ1iA(uGQQam zlHT*u=Y9||eftme^(YaX*FPDA5xtY;lT zm~E!7oGPSb*DIdGc^11>X}NtW_`I5P))3@~Ls81#+0MDCoRv1x)0-*FhUHs(JDWEN zbY&u7Azo?V3H;+TWe_d>nRN!wRbX?BvKUgd{H0q7s}(2JWm(hgjTDIT@tznHKD`A$OTkWkM943i`Ix zHlg^c_G+dOZsKJ)dC!6xOOP>a(_FVIenjZTxPd6ud*i{d{6F@3od!ruej`0B&>H zvRZa!G9(Nc|8@qDZI*XAR`hqrvD6X|L^iwf1`s!d5MKHXy#$m1sCO&UDV3Nb+hXBb zSd+dU8plhTG2JP-9g%ggMHVP$L0E|4B=jd5vH{Ew!-=^VY-QRkJ+pmNznBb%rIVHv z$`!`;_t5Y*b1Ers5d#W1>-SuN>Lls!pKsQ;j>e#Q;4mSE%`8h4HJoz}0MsZw6UTy* zsb$S6qDWZ%rC12UEik@!?uY7+l-&8n$?N5Z7@3E6;eToNE*0PH%_ISiYLo7okY7({$D%Y*?acU=X218&z(vgJCl$lW>rJR*4*16}U(^ z+s{!j73o5X4^^mpxGZ%>VFEnyo}|O)J3;R38~xd<+GUDdjsS}kI$-eGMz>Pa8v7EK{_!dP^q>@fv3zBhW$xYQ%O-sI)P6In+ zy;7Z^Srab$_#!&Xneh{Up)kW8T2{-iVkicJ9ra-**rj=3Oc zrW|Zl(*|wTk>|DF0GKh7$wc^2{hKB2#He@qROXTF6zpNFS>P!+n?DupwykNg`yJ+K z>8IJvY~Y~xnK)663FSRX`mEd{7)eXGO)IsXe8oad$O&yv@eZZB5{a$FRZY2 zAU6nSFkw+k+Dw>DC0*{!jXP60BeX@NPkf7UI`J0jZ@^lgNlck7MlhqwK|}Zu?oZ`8GOW)MqEMBBB!j19gp zkhAjJ)s^!ZWU7-rq@17`P>mB}Ah{*eTr&C99+w?GJYxls;A8x}0Qeqyb7>ah8MxdP z*){oMIJGwTIr||b6*Oa!l7tGxw`H1>H-bAqlA?{Vl-T5p#SjbZWi9A9{ahAY_~FA# zo>F16?8j||Svutwihg%zemnj;&vJJp_GM-ZmqTr5O`Z3R()I;&&jHs(cxJK$Lt?4{ z(c!Q*vMGY~8prKy_e=t2_%lb%byrl33>}EUDWa6G^ar7UjnZopS zL*R^kle!3?@;w{R%>Bze*iMung0)ri?4j+cEst8|=)GNE15ly_p>sqkD)4jbWO-!! zb05mbikwIRFLjwRFD-=mZ7Dg73&rVV;)d7j;HG0Y6dMjs6N#A!rz}1KTnhKAX*Sz` z&a*epLpR~e&?~!HKQo_!DZT1Q3*)Jv>K{&61{2bX2Wi38b=#k>+N$vcdaVZ7GYV}- zk=u|M(cEJ{6LF%2|FJ7eyOX)Tn5K?Fxo2I?h*K6}N;!^ca5G(1-R_-P&cRL;dkus; z!4t-_m90m=X*OzF!mRzEe+G|)j@>7t=`K2tCwi~X`Ft`AGXf3RnPvW@8<0OyE61A( z_I#DqORj;+K+X}#;;9R6JC&Y_VPV#_M(>VjA3br`X?l}LI|EN?3p5N2-=5wrKdc|T zFc6gS_Emyy@NSzV(r$@GZ^|;px2UqZ_pvwT|t;MY9dbg$MB- zYp5KlURU-zjuy4_zJk>XrZ#K47|vEV1zM_!)31+LYe|<>;T&s z3@^*MWk|*U?rwzsziqplgp2DN#F?R9=t4h{B_S-;SJEsQ7UJf%v&Ssp=4DJ_pa|O0 zfZDt5&IZuZTr!O!50@~v?n>Mjk&kwrBH^3VC#^E4nj^uZN`y$&{rNmXmE}{%D~JP> z-h($NoBhrpY2co#A-aH;S$&=xl9@f|(YbW)8qZ+J2`hQrlAR}L_gsi2!l)YFl8^Klc1%|(WWKiO=JO}h|-4kLp3;CH103pI%Dfmccmk2gU zbQJIdfZjTK=~fP;=GGk83!x9EFc^?#0A2*RAX)iNO>C)45$DCTezo8eVkZ{DeFScV zTd>_tlJS0Mvu6`)@*b-sYvsQ42KkIARZ;#`mh6V{?x1nesH!%z84*5xmQvoos!43B zIs=Tz7Vs%z2Ypb@Vvh;a!%FuN&FW`fG8frMB$I7PG4F7nhB#Hr^&!k%Ig-5|P3c3a z3R^3y5xY|CY{a_`lo?p2){GJ89l0Hv(r>hvoslFNIJBD+YrA29!Il;sdL zbtG6}7N;e`KzQ*(2x%*#!!68g{)&C5SSSydQ6#bu*Vi2yUMqqAkkD zWMCBehhTVxU}OecjoHOtH+|F-AyUVmS_=L{!4pR;L-Z3zyAQANOa)U>T*zjR&5MW; zCsvjiUkLiZwa;^u_<4#pb)&ssJX0Tq?;o2lz!$&)s8YYe7DVQ zj=C$1Y72yKFM}&jfDQfClVm&B^Q3LIbErqd4`uT{9GbrKd^f2mwMV&T(9>qu1r5e#AM0vDK0q9I7*d7HY(VNyLidQuR%v|hYjZG zwVr6!eu;eWi_(*P*6`{B>A;|_-5SC)83|=&Q-*KiM ze$iGr?Ixa74Ly$^&w`ek&8(0!^>~`TpLLrPPv5Lkel8U*&9840< z)I}Q`etm!I?v~+(!!~Qco8*(YfV%<3RIN4HvXs5uClaIY@b7uJr!d0XR)*PgaG0%D zjfPI*tuch$+AC{2YSy+3prKant9u6_s&R4XQsRFXYL3&cZ}jhr(>%1 zse|!Ja<$F&n`AHJz#J&oY;!A&IW}O<`P6#X9Ur&T%f`zm;eb!<>CWTl%OB!4#8Zq0 zy^*CH{m=fb5;NA!-pza)b35No-6u}6`$<_YKlDKuNN20^om<&a3+z+QKy0)6o7Qxo zT_d86Qn(@K1$ajr%;2(kU^BVqx<{=`M;Xh;GYjb0fs$UT_f(ZbJZy0bdz*Z1f z$?bgUakOb&=MTVpF+UDLaOKP4AUN8Sf~y*M4gJe6WAMmR86(uB!m{iF5kH8ca{-{jV@9efU@U6ad&vo}k?# zu)zW&_xsti5jtH9|8Zp6)k-+bJ`6}ehc0Yf-d&aIDlCc17J2cB?cXe*+2DO>=;Z6J z!^SG*z+r$7ietk=f1#Wqd|LVA;RHKw+yuCRzD`Oi z813rr3_I%mpRaIXS<&78EkT+{r9-%*XerD?|Bt&22kYYnl$G0vP&9M*W zfX4By`Y_Y{rLZfejvD^zjXz9DD8Y^*6KR`%WDD%=0{y+Tl_rM04~=ZxB3L%^(l!~S zQr%jjG1FMFvBW;*-}}3t$^S%N-3IxPCyLb9(71xBGKEdYe*Zo>(Yd_)t zy@=M=cr}qKacj_4s8N;pn2~;irfPb9|317cvuE>ui>MgmAw_*kZb=tBf_d$mA8yO^ z?c2A<)41(P8L_1N;x%Exyx%XtLxK~yjp3D7>I`O@B2iUTpbV&@;a;jk)n}&4xvyi; zO0RSV@7~P~rf)@h#YT27p8l@ds-k#1RX4x*Jfl_Jv@~S!Fi}n1acZbSGhLcAH5uo& z_UV&-1nCa=I69H8U^nnrvCDBIaX{js=~PaU*5`>MV<+6jcQGg#%k8$qg@o4V73X}< z!EN{A<9Y+GTD!WT;02F=ifQ^MV&{vBbwQi2YdpC=axO6V9NFe<{X}Fnq`Y=fy4d?M zR#KQQbnCyhZx-R_S^dPr0tvsYyAbsDmJ{m%l=#5RkgHPu?T_o+Pz+Wvqh)d26RiH} zI+m?~80@3yofuj}wK{#kn_cqSnhqDT2u-K*VqPpa>%Fr=`YMwm7GIRZBU!0(pcpx3rEd+_H#j;yp*iZmyhAtzgmNM&F;Dn_3 z+hf$paR``8)Z@<*7_HTOD_HsZ!)YmkDSS1>d`K^hlK&%m+>VmYgWO~6msc7+QQT|YU z)lp+hWwhp0&@}y<_)#j~&!bF==6V^gewJ*6z5%OS)D57vjUB$IdFy=%w}~Mv^!nza z=9I?4_rPb|+)@@VyPo>iVX_z_BCn*HKJ$aDiAqdVJ~7^vGJ~Pci(e0E+G4gFYf4bw zk-z!lbx{N3RWr$WweQt0bHq&dm=y2Gh1Jny2=2+)!6H2F)(a-UL4=zbTdZF>o6o1{ z$H0YA^ShFV#fRk?@)On8O;-P!o#K?u(&P9{${%k|dl9r@y)Wz|W}`4MaE~e3DM0zq`jwaZ#;0Oq6IX{6I=F zO1lNU`EsmeLR@am=#h$j5*2}S;HE?h^X7Ii*TgmXNoe_b*{7L!?|%^8?Hb7?xj)tL zBXC;Uy0>6?TefUJ0J)zq;TUPWqxD4#5$xc)MLl_+?gjYvQiI~u&O2^LV(7BA(=T$) zzk&Elm90tsSISk<=c^xPJ-5bQeU=f`OcDJPHz(%ta-`0*eKj5-lczBG%@t8!MeeM$ zSuoX8Ukc@12JYy9KExz4kMUakGT}AV0%I4YPrMy<$<>e6M0=YE0ew zvb^m6P8kHfW`d}u6q!{(rz`Z2R!0@3^Sz8>@T{Q@k@R;sHJa+HzDGo*ur|p&lYsLw zpG|)B^P4N5-yW&N71a3i4Xy&c0j1UYB4zu!okx{ho26(l`TWm7hbMAyq!-Gpp@S)Z)4;61n?R$JFv!Z|%`2cUerl+|l}uSv33#xjDJC zY6;?xYsFV;=#>blMntD2MP)RKN-d$NRrgJqL}rSWQhd8pE$O*%WQhuiB_>Hx>Gh(% zHn@uyvKW?WDaM1P9IHuZjqiYqnuk0SnlH7CzBgj`!d8*8vyU21ZBu4W1~Km*&C7_s z0`8ALw)@cEE@zMoIloA=V&yz?W@UN|lPy*r`R#eNdQES`wQt$=JJQSIctKJZKh#_C z{q?=1N8D^v(9Ojr!*a)t`qc&ccU)f*i!}zIIXWZR)&K!{=I^Ai9|`I|nk*VLia$v2 zqt-lnf`|_rEef`!UF-iRGcd%o9Sdq7!*h+UmgF#o#u9}|Y2F@5tA+U|Dg$hDQmi<} z0IGk1UUuIDw)--=zY&iM6RSdbJ6N`QR$XE>RWq3DIOZ2~-p{Yr0%o{Z@&(nL{Vj~1 znRgzF@~|NtdRGsY+~aDv`CdW29VdS=ZjMN;jC>-^XHU~BXjk#*4AZsyI-1CMq{mew zC&ur4(O=iy+Ti}W{j0vRph~B@Vd9*a<4Yz!jG$5a<r3pa)lQ{#cPqe%n?H3zEoQz&mj2^;_k7ek{XGxx zbQ9fAEO2w<`e&}g_rWEwa>uwznhlQg(wBLcbShPsQ%*_8n++^E#jvlvXC+xFJ>HvY z+^b)#7QEL2u@8FYHkae^m;SLe{oQz$aBAznev+JhiNuo1Rfvl!73B(?Pgs_K+$4;# zieoyOX7$#H+}5gmPSc=@{pkwBkvb>DU--mFyN~5i65EiYj2|Km4zdAtSygxkFJ!udH4BM1UWRh zct~+BKRoP5Ludkvt>){$Tb*r$chi`tU(idaIk`Q}h5p1`UDR;EUYGkhsT$?cXWBn( zu*96gEyIt5XB;qwL;UQXuhouGT@Y6R<3*1@=EU)Mhf8+eobTdkgQ8hf>=i;kI!6hv z8j4k^F*Z#Kajmw`0cnM7(D%}?(|6E}UPK_A3x*W*w2Qk!n~P|dpeWM>_m zvTDx$VPVmH<)bgcQCgu)#G2t4y8ONr z{6o$2(YEdA6VjD4E23(J+1I~yh@yQVMS`JqP%TK&i+$0TgV1ulzeiTPMq=t0Ue%YL z-OqY5mEa!7jm)D&{3#O}3(Jqw37grgrNWm;EDm=P>~LWCEulJ7S&-Cc1@&cd9|1a#dOA(=7`BI7LvWc5t{Q#{Qx=NO7-F;J zOG?^H>+dkRqlWf$C{OH;#={m(FS=E_fTpMOCZB_eY`?g5v3i>rH+b#3bhXHrDd~^b z5?_;01d(J0?A2kQx&uAIp&pw$RLz3>8Zq{qyuLYTfOCq}A1kyP$|SRKPzQ8f=^Zrr zdZp7YG+GB7qi)j&OX#zrc>eyM<;2hppert3MgrW0)I+aR zfB0$kpWDjv=(cp*+| zjs=tR`45_(rB$r_tRDmDoXG`RPkFB;?M7)HfQ-AeTwOjSbgu21X}DCM^OZ=qZHN}W zU}nhL6S^`q9d`}9D}HnDFy$SY08@95vN@-~Pg1vIl(vVMs8IlGK;t{^c&blF`J2%- ztlBaNd;XW%#bd4IzoM`TCMC{;6MF<&PSyXKS;}s5TS&xH?eT0_Oj`mPbAF2)Z?T?^ z_L-6lck3&i%Q(TO7>IGpK)5ulrnf=g8XRF>mYlg7Q4yEPlE@M>Nh37`(UpcDaU* zdbwXfE~nzV&HKr_#QI0(%!uq|l`+rJI=RM!wKyW^d$h3Jhqx=RF!rAJnZEoTHa4di z4T(%iY{_+2GyjM&3~FbN<?l<~&ylN>D z6f}&n4MY^n3^rd7{iVYkYidenDPB14PM)nJ7Ogk>-K(4X(6-YwJ6!o*S}?COzC{Hg zpX%iX5bJqz|5GBkR@hEku7mGmAKk5nlSYR3b6F?dW&sI1mn-#;`^o2kJ!Tt6)*nL~ z%P-gB8o2wJCsnEEis=SdC$s{xmo3bYSdDthe=ANY5p$$ot_tj+?;jX$^yydh`e7f7Y%XP-VW&Aacah(!?T3h$GRPClNBSe zty?`lMZMg<0iGiNWgrYbwn+gOB5yA*?b9xcB4Cv69x`4RZp~ke+qjuixiGAwTvyPr zMRnJO2^$~!d#m?p)YHw6qs3eL9fnSLE!K_IzN)v3ha@=%y4g$jyLLb6AjZAhr2iXW z*(dSu&}v3GC1{Gj!Z8MyGDc3j$L<;*^EbESVhb4yvoQ za9w)7Tnr-1hh`>Mvf)qcegtisufzk*y)mB}?`?Y2gdUanepnz1@&^U`X2gW5Afb9`CMMR z6QY46Q?IdXQHkKSpE5zP8mizFo9UkKf9odju#|*_0+|nX0OeLH9{0nYHDW7R(%KTl z%=BbfY;qh!C4u<0xXSvhz{!nZlyW!u1~*%$40cQnQs|e!TplT0SA7QZ-N<^pO9{TS zNRjbk+Kl04lxZ?V#KP=Q;HZDWr(!Doh4ky#6^=os-`jUL0RIEs(%t1edbD*J)M|4R(!_IIwc&)>T_T2Dsahs^K;c^CdEwZ`QoIU z!QUF+q@y$aIMJ!yK~hg98Wnaw-*?X<1w+msFtuzW!@gm)p*cZ!6vj3Ql>Y{Oo#$pu z3&Xxk89a*lu~p{zg0{IAzo5J+8tUE<^ml!JM2^wpY{BWfb--+EY)C1(Pq@U~N(~RsdKeIoMnEX7>{5|;aw~E&J8nN~b zODs$}^=i0hb-!=?;&bUw)1Phk0&@FWyq132Ex@-2Bs^ULE$pS+JZCIuB&O%gx@@cO zFZ$FvY#e)BpfYKLmjF94pRbma7isjCt^^0v^JxMWd{{q^U!6I)kL({@d`r_+*N`j^ zOgUeeJfkjMjCQTJ;EwYh!z83_TZP=nL=76cyrq}TB5lmC{)cJ+CT|p7TfoJ%FHB98 ze=ljBFfiz2*UB{7eq&T{0Jh!3FILyb615Yp%*2IIy@*yAW&KX;y{&gvn;5p z{-fe)`k=LbeXs_T$9!9M?smkhn`fP`@+3!otM~rY2JGH_*`P3X&!bi_V5jp-_uZH4 z3hFiYJ}rix)O9wN@&u7ex}?jzuITvVh#pPaFMt2 zbaZa2tkqZxcS@RumXI2-U;ngcrIb{sURgHUWXLM9#)+B9M?Pyp(Uo_n*r&VRueK&8h zc9I5BD)0uxqNFJ{2a@UN7u$OMVA=%saS}sO_ulv}UiBaBIb#ya>B-5xqd(hzAKe>s zou4iZ8sYZlWvISdc1Ecx7qZHDsn1k9r8XXPGIuZd@2Bq>SJ~^1)Cy~X`csn{Z?5sG zg?f!vqPE)Qw!;rI?mksr@B0x4gpD#gCe*oO9_!@$3I2LkQkqLwWb~y$m^Oi3v?h*{Vm$h5cJMtTz*&(sQD zv&()hBn^l^VEBhujwtP}8K}zBHTl8$7Vs{Tn)R-7gC-rr2RgvPOqIjv@Xqn(=qQa# zs<_jw#HEo8@Os+DvG7wlR`jbNtQn#y@+)KJc(J*Nb?GglRBK?aQl{j#F2fLJ3K)FN ztk+0cc;}Q~S6cR4Zx(5R_Six_A)HK=aLM2^Ym_vtnR5Nv=b6v4HBp?(=DyXs2?U(R zKhgPqyMSCSO*p-g8kfb@kDM_o3xTKe9v(aYq+V@>a$Vkh3q4i9#`-J$mw&g^_F-Mc zy+Gh}l8zkX9=!TZ3aTCGXg*!=x0BzUuCS3lq>|`h!AaM9ZlRd%k1#Hm$W%@fOeiZ& zYG=rf=!c!^S?BH$Lcj=3dDnA8vAIpKc5w4*`N8`Pa=zd|!zvG^^lcf=^(ZLPb^ zJ@25V4EApx*-E`O4%vtG>h_jGgSt!$XCUP!-x_#i3p#v-hz{#p-&tM}ToVi`(n8Q5 z$Fl4R#8o0Zm0R{irm7vMpV>{8m`YPEHI`^eCl6-ve&&RqGJjM_{6)STth^L9 z05pT8I~te;W1kXa*BrQ|8MZX~dUbjxui7vSb*}BTrJhx7N6w zcmmq9SbV+n9x=bNPZ;9$ck8%Odfgo%cUL+?@OM-8`bi0u=lA;UNv8r+oAzUh%A-dk z3qap!FEI^MXq)&ky|nAW=`oQ0#Hp#Z7@rk}B83}wO zxq3*C89yB~y`K=}x8;q1l@RJKEa)C{MKrc_ZEw^IsuvQ^7Zjyf2h;3^Y8*rBohT=Z7R}=qpY({b2=8h}M14QsR48UwG4b6L|A8;7qFz0_|uZ)dGg2S{G-O21BYWp+>kK1UF0yLJh zw?yNnjWeNSgL5gJ%=V!|OkQ>=A%iDgTdT0UG-z&`wFV!wh?$JLx)w5tN5jZ73V$ahYHyL?~I>jS9w*DrEk= zzSLo|M0I+wl$J5&IcYko+!1ie2RVmpSE+Yk{HRu&r`P*y1x)K7iM;Dm zK{PFfVYxs6+}}_4Ms31POU;)tjAuHOs)uZ6>qh^rLD2LE*95*mv+a! zpWitlp@8LSgO=C9nTd0jv9i_nW>Wk`em5AUo5z4s*os_}-z1k2OhXn8%c`Z=iS+%X zSfrK>GovW{%!c^olF8`GdJxgo#1eQB@q-~godVlHb^AN)1DPE zqa@vEE&r_yfBK^<@H3w3(0tx0_O~+R{$V7BFju!vV<8!Y2)p!P+6uFvQzeLhC~r#8zN&Q>sTgx7{00Z`>yZLbFQ;o z=Q`(kp8I$I?)yGco57SS$@*1xcKrsNxmUlG&&WbTFuCr$aetpe-(K&fJKQS_) zudjD8(O+eWb^6f`N_s#k%VJSh1gigQ@e%go_?dqEb@Ai*xIWNzqm}9!=-K*j?5{`C zJ<%c6!)#8v03SYB%e+xGPfKY_(LIVRfqG?WHe@pwc#JAKtHhXHsP(9*_;qAL(s}!T z5zu+{;lG%Ci3H}VuPOi3oi>n(h?`Aj%aRczL7NYEn=hF-rS_ou-+`^Z3MP#3zS`vY&7N<`mrX;1xDLKi&;1W~-oxy{}*K=H^_MmMHTCYDnwP7_#hq6Q&nWrxs*p%gj(CTgp{*(^M zuLcj-*X;b;<(Y|XFsZH7MvNQ%HwI0YYd=5B!t}ED`L-{f{=2ASTZ+EWP5|2Nwj18gYfYYpR zVIU>q&gSiwqg;q*pfm_1fBa7tKu&A8%*BAGhfH0!8Z4V5HpT!QdG!b|zB}0Ior|XwPRr7epR@LpVnw_?$dG9Ck zfB#;;a@{i3Ve|47p9&eA1G_#Z_%L`!*R)SPg@@sJRY3<%Azzqj=3Jg=;CIqpEa)@G zv7r;S7Y|R@Oq_M9zP=sm@$u`(`f#>e*J|?o@psNyE(xy(N?SMIT3K1;Ld*-a^M*gSR{i3vz+xW?k6&AudCNIm zU|WvkIh^wO5tl7ccXTSRxZ{=!=IT!x*3@;KB!9#7yrpF-GPf`PundozP~ZKU<02+C zAF;j>t0x_}m}!(+xWAMm>QH_2SM9FdJsS}l2Ym|Il>1JS?z^Pc)%J8f@kV{Fdh7id z)LgLPby!#vT$EY_RS3*GGOy!Sx}exlHbp=|F*zs6D>*P)*6<} zQId>>&W3d(!&Ol)W%T76Xg#ayGwjCI>QI_;%Qz||BJ;-kRATkJ2ar=@o8>S*tz1v8 ziS{pIyV+#keN^#Kp`{aOS3h?1&96|5mT8maJ{(tOG2 z6JS~rd1s%CBEjolU2)qC*unxSYM?gC<;#(MJJDSF5xL935Yu`o`VhM>b_LYBX>c2?hm%F0 z83q+nq&@(+P~`W*RNN9z2TwpJUyMaF9vygO1YiUgqy?Wt*Md}#vupGOtuW6sKcaS$!BXQcG6Ob1B-n{+@ z8s9w<*G8VMa1hCen2(gJ5!>fx+b?Dl-7UK4c54y}wMvjRBE)oZcZHx3o~HB;wkBFY zeV8_LHW#JxJy>FTvKz#?=SwUUF`$Y%frdt8l11!0PY0Dn8@!S~Ca_p5XhIihKFRHW z%vHjKMGClpIF_kR>tXb^OZP3tWAicQZ!K+?jNZZIYJ!S($vg~uI+s@>V9$2d>C0vVOX6Pb7Q8}_bVWfR{fECFNT04p=N=ZmXKr8n!c4Nh|DCCu(3K_mZ zhNch2YFed^obeKXx_8Tg+tOPNmnipBF|FMuxi(zISKCkgwn)9Y^V6`X zi`~M`AOpcmi~p;HV=uYwGG})*UI|gm$wk?906_AO5A+bUimEF#OB!)KO2G87LN(GQ zHW)3hFRe`3cQGN{vP%=X=-N2S5IShILj~F-OH(Ql{1kmtIJ=uPU5AdKT~|%p9LS&h zhG&$w*&1mXsgqnuV+Ekr85pyX>Vq$6?~)!hkA8)ZSQOVxbOko(=R;K<>irP~VMxpE zP*z!yN&r=_K+pIX!~ddO&^deeI$xGAX(?;-r&;Fe5l(@&U^hx>I3^DzC$f4qq1qpm z^!2lzDC#N2Zk$_Gyi48D;x71^MZ(JDhuQzSJMI#h>9K)5Nl5KbS*z4#UDYkows-B% zvQG{!529OqKKNYj4cdj_?fQ!=Y92IH#)mYeH(ILBfwQC=dW@R_n&slhQy0G+U6M6$N|iqBExH`#Aft|eH#dY9NmMnWD|X8bISW@)=s&$HAX`$1E2ZqaL}(0q zNt82gpb#(6ik>tR0LHi2m*lpAOZ-R;EU>}hUYkTFHL6YuZtbsy#*;*e2hB8`=bEer z5pk|C&U+}F@>y@?cWXn*Q+G6a8VGWN%F{{S;xz*5obyDKU0?Z#IbK0iZhYgwchCkY zs6u$4;T9c~5uqF%uNrG%4UO__r5a1QFCIaBG+%yh+Jd++fEX zL=zfr+;So=2v`-Pm90r3NKm5QZNMfvh~rdM&`}^rNd*5kLqBq_bEbEe)A^GBluWJ~ zF?w5t>(2EkIyxy5!LF>UU`T=@Vq! z8{wx167&5sM)ZnADMq!8?8`)naJXIKzCg+q#bm@h>E<4<^5!<5p_(-mv6d7|m1dee z4N3GE#TC!3&xEe&CU5J8h@Vq3wNip*iYl=xE%ZSvj(n!^f{(bh zK319qYGP}{%(tcu_c6n#s3{F*WL^PRAZ5|;1zrTcA7JW3D{Wn~Xh?2~1wuAgCK{fcX10I%3QTT{Ib3DSuto1;|#!77@pBUSkzPHnW>#BfR{T^WJ95NES zT^e=t#7y`jghNuJV1>+r1jWq*jYbK{82@wnt=c}jNQj)g45CmFu#P|%sX)Kfp^*q{ zsRlD{#nn5cE6H|qB_C*hSa(?{YWf~}lPr?a$sCPL$Q?T1o_8yO2Kau_1DK6(GSC0bb+ z+YUbI{ts8s3DJQ|b5~mYs0dwl6GolU*BWRfqK!~97eSOn+^6(=P`&PU7$qL}jp`wu zDwq&W?(Y0mjpu50_~7y0u=uhxNJ6T>M&n&B0wsuesu}Gbmhjxv5m{qSx+!5-7KVt7 z-;Z3Fc7Whslyu35nM9#G9^pZ$FlRdH-OzlaghhQ2MpDkwP^ghAX$FaSlM3yIQBjDJ zbq0Rr2!fPG9ZKBhJ68bS9()+*mQIamD8$J6?Ine4#XXrdOHAXw-&}aV?VojP3Ot%$ ztX>;hYMS=qd{D3Flu(;eCkWASU-|UW?9tQj0zE3v_v5E-02jqu(HHpFY}%25Yg>S~ zS|^cJ;&6phcJc1triI+Y{H)uuG^TGvxuUt%erZLBtHo-Pi{Usef-+kCTGNqrl6D`3 zEoReQC0T8y%D0vHurGfn7&wrY`x58L;5uM3BhQN;TjY6byn?mMS&B6KBug&-$l%_; za2z8OJCU=r6EpNo<-}W9omqaQ)hVBGNJr=GG;*KFkFg@P{{-%^4DO|1T{uSRcEM`U-8li|$LaaWgEpUF+$ zmvcn~&jVBWF*&E7U2X$%i?2NkAVG(;*vPc}#|@QjC0~dqpUC%O5~lQXbjr&1kLS9+ z@B+?H95!$(w}JZ!k5vp;WqUY|J^50?;_ek<(&P-Z}IX-GS^r+{4YwMclKjnsY+HDFy7aOxPOa=k$ zBWOm&?B~_9KQ6rT!0Uw^y%;f%v!_28a(p`X27{1Wo{Ui%veQ-AgcOx3zrnSwr@~<5 z3|-*CYt{p{kR9|rZVJd$L|kxFoqVFr-ZCd20rKASO;BF@2$MHi$+w$BW8ONf#Y1CzJA@gHJ4sOK;@tSWNo|l_IdCSs8 zTcP)=YE-7{|CY{dA&In+d#fPd*<+M_Xj4b%r*1ka0Kihk%GT@+B5&YQBj{E7!qsFc(Qd3@BURc7k!f2`=Fa0;4`@}@!w-50 zbRM*t&K0zqQ>0tYJuXtN5e%nGvs*UtD1r5CT{~tfTBsee`Y5`dG8VGO`j(bitStR; zd0C|NQZ?9Fk(1y+bpi#@J(Q#MY@+>EcGLi6iw;E^uHoyz1=X!CpTmlLOSb2 zGJQ>?wS1On%rJiZ8KD>L8A+g>lz=ve=dJXUKMw_a~C7@$IyChpc37$^br3)+b@?X%i%2a)zN-=p)~PB}Ag9QW$JMOz<&xt#3T@o}l_ z@{W*BMpU9VmnTvKUjIs6TeVW}(`TNqt|Es8#H%m)UKk?v`49QTdua0>*? z+bk}sF_D4_0rlX+DHTwk#e9m|tD_-1H{S%NHf26xSiZWdPQq~`r^;}=_e6~pVQ4k+ zN94yUI-Qy&N_p}llJaO>qa#dhpuhl!uyH%-1-kk48v42jG-r|CiH|^m&k31j75hh; zwz={xpt6XM&@W0Z5{cxMQi>1N&3_mVK*&Ji9(Gg_cwwqBL819<5r=oJ{B$H2cOQJ&@+JE6ADUX$Z zx}f0*6x0BSRc@QWuks@@Tk&{?R^eFdhpyQ=bxG5LVF60a)X8Gs?&{g9;2NQ?BduIL zlpT_(*q}3Vo~wxxxxXSqLlB`S`%EhHGJk)lH?XV2F-w-{y$r}<04Vs)&o)6&@0U5Q zYE$FCQN=V0)l@bp8{sq^wiK1O&nSF1Me2DPRSxMY$O%?+LXLApGJT*LudvCc+tcBi zMu1VVb8V5v4G*;|&`^a~@&5AHaqw%!b+;OR!Y3X|^V7eEfKN}Z=~sr6m1mf7sr{WM zDCx>ts0#=fT}Jcuwg`Q?oPDT(8jLQC7Wq!VQa%$#ZZTxHz-epK#p+{z7=cK^Aez!O zs?bV>0jz-1(U?UA^aY_x6kE)I__>* zDr7BCa=nX} z=o9|DdA3Y^`uqK$?g9b7b)mK>oe*#r)^8oOG|X(-ait1z5i+PIR&1^>>@u15*~G19 z2VV+CqGTxeZ;Q@s@_D7QpJ0Heh9|NV+}X>vadfn%W#3mK-UXhK_NRi6Y+Il8pNBUU z@WD@iG;Fh!xDyl2Qs}y44E0NhR<$^-(IpCs>iiEHT@09=?cKJ*=n!TtT}zi}@VrP= zqQ5dnUiZY;e7CpKn_H zy3%ty&mO)@@KEi$mFs-hVkv?J5I+4muNz34aO^?}s&d$9aPRnvT zZ4-SDyQ8FF=hEqCodCA@Sl>G+A@M2c#JY(ow=?@xkysu?@zlb^^Mpf{+Zq@l!~oXz zy6GH{I2x>liP8bzUh>u?2J-e`8h-s!n-F%T@Lp+Qegq44iG67G7`c%fnwJ2~E`Dkc zVZ;Np&Ifw6XD2`0^Ui0z@8@2sH|w6{*@}gcw!($gO;6-a+-A6FhRl;THloWW_wfqMBb-`4B>GT8 zq}%#LJt%OlkQ;f#?vu}c;45Q_X)XKf=fwL_ub%L)qXxN9SIS7;!C4G?Ba1Wa_OG- zq+!s9J1o4x^5aOR^0!5|Q z0r&FN&Vgm;KAtZ_|4qQU(9M4q-a~chH`3XNr?g7V5zHotgBJ3@x52ED_LFCQcjVlFjOoEUtdGwZ7u2(=yhkW=|0*< zwRHr@hD)tu%XZCtEo4A$(TaU6fV4{n0(G!H9)$Hx4Xq8=z;E}PMi7CAS1lB641`0^ z{Qj+Qkd|CI7^P@p__6FR!S&mNI-q$#bG%4)U5=^mqyzLth{Z<;FzLyog(d?5ySBwc zI^t{L<+QZ&g9S%LKc!@_?xdrIm~F|6K1yTQ*m<2jW49CxhZ#r?HHtm%{4JPijgDXrsEExwf77V_FnTX^Z!n8^Gh4VlRIRK%xlh9Y$b>0;1r*s)=?oP~a%jgC8|5E^+J$3P9fiXV%e*goC8nXZZ literal 0 HcmV?d00001 diff --git a/src/main/site/resources/images/backup-intra-cluster.png b/src/main/site/resources/images/backup-intra-cluster.png new file mode 100644 index 0000000000000000000000000000000000000000..113c577f63c5ea440dd82f0937cf0b2086f04ea0 GIT binary patch literal 19348 zcmbTdbx<5Z_bv=sB*;Pt!4``kfyG%I76=K11PcVW#o6HQx=8S#!6gKDw?#KVaCd^c zF7EL0zTf@b`|qt=Q!`W5)iu+nr_b~`&-0uRB?TEEJ_sKJ0|O}g@xAinefsene~R;X zyW#HIV;`VK_+=e%+svszX~ zl5B$Ex0vIsgn8__f1zz@nM#vxSABGukI<$I^l*evqFX(AHBpudws5S=mk}M6k(eN& z_6l!ElKE63FV{n%QHSFRM^~WCBuf8P`+@-O?CyhulUhL@fFb>zrMj3G-4Wb3u0W)e z_~nB16`EyHa{fqkg_eWqyM}>l!YUhzmxi8LP-W+>VM5sNsT$T5#y@ww+*g)_Cp7*w zZxq>gK#l$GU!3=o4MsH@6_p-__-y#Kj@*y7k2jd2R+^$F^jx~fpZSzpG|2wpeABXV zNyX$tcGTRZ!x-VpDRG>X?D4zfsz-o)SLN{KLP-6$%fJGq7W08Prg|%Tw?SxOP^v|v zK&Tjkf@HC9>|ONc8;N*RhhqKj9=j10!>wEg>~b4g-sHR4j0ZBdi(_4V-#woBY|ZXq z4QXoRn(D3J2ZAH@i5#9yvR8dwgND3s&g_ynn9A#_XBS#pePyrBqstrl;@y3W5l+?j zhUgP(sgi(H>g1F@2k|v%4QtlWcxhpRNi*$xPs&I}NG1$Lcp$7%j>(h%Bbcy-*7l1E z03a8qG_D*+8Ior}WU}wJ?u)YflL#!HO5&4-4=s}%1kj{7k_PT|Z@y%SkupvU_mx{< zu!%-7gpHEe+m6-Jgbf+~n>js!LWaSsw21zObZPw;hs7SF!(~3F7vvT<(W!87hAVCx zil{mfaCBY9aaRsLu6W0R^r+h1c`hef&{{B8a5j#AP*hO!J^EU#Ca6{Qqmi}gj}cL* z^p9YQzL$C^6Hm`!+XsM~b9f!2;ehZIDO`hj3pXd2QE-l4HhwWqJf3WwU>#}iqWyEu zS@!blrP}3}ONC3lODf~#7+cLt+4|R>Z*;=bCRR6%PWBM)+7mqTO?oO^IpNjX?y|j~ zI6szs?EITC&JL6{8l@ZGTM(^tMY^)%q_#CrOq^E;micP|6tKx$C4<=otE}#&EJi;E zeKeRGC#ur4=KhnKNH+E|%%hXU?jwy^eSAheuG5?P&04v%`$T;6aI!Lomx;psMg8ut zhTSB*+-J=w_Vf2s?ILCxIS*Chrq&_eh50(=r|sUFyz)XyZk38dO;0k`O&wN0pSI%I_S{lvV%WJpuUQVlz>vU@eJ}ad6>~og zH&yRLL%ms5ie-%a;j8B7pYYiow?7pA>WKg48~UO-^w-wi8-Kl~X=?tq&tKSlv9W%; ze0=)s)5pW+{akDQrW>F1Dv{;BZtGstG5(YL<24bp!PF$7!=wu&uY>7e&<_Uq%m0T1 zwh7O?8tfBZPlYJe;YdHfb*jgAAEz~`$}0IjLgg%teJI^R(eA}X)YX=;EHw@;t%4m@Vw#e01~ZYI?P*-qm?rdpgz8Xy!!?+jHhnDJw#TbVL{-b0 zt|Yyg_>gKnC5&F|ax5q|W4YL$-~r+#{;f(e9`kQ-ev5AuaV}!I@j*${V|rt{=O#Wt zYQ+i45Ch$S%+gBI`IqoM&42CoYt+$QO+;*;*@#o9WswIf5$|>vDz^XS9!Rk`x(v5@ zu&fclcx^sf`9*C`F%2eV{!GLQGa%H#Nt3YipqYN3Ld@AdlBwPVC=HTVYT}BZkH39c9g;1Cf*yMA?C0gG`SA`Mvm)!2^X#CgVm+aFwrlKNRKwAi%kO@ zxu6S4Hry0>ndG$(RvYJ+@675zcuX&p*g@L7Kmr{@#M_;va{O|NK-+bw};E3~1t1yHk`Fn|PSx>;%5QvD6; zG704Ly|<;lw7uf(4?su~Hz@8OKKGtzzt4OnjyIO|-1Dugw|YG_Q%8j@fbzk9_-%=e z`#i~#*Kqx88|!qOf?%q#`na$3$rFv4;Bj&^jW0;*Pd0*_W=_e zzys0)0FPFO$+e_xT=DgAz|N(fDYW3NPNgR)F}p(fe%|DeYW8?f;#`vL4<+b|J#O|s zqbo&>h(qP!#36*?_hT>q)33pZt`QOlezOwIntA6rDs!aZko;MvXRqYH~b}; zWhd5;raV=~f1ppth#SeCA58x2OeDjqtg5FkveB<8`E>~vG`e{i7|)|H{_K}I1BJhQ zMsqtpgepDo$d>iJP+UZmL>^^Z*+9Dt4CiwEkQkIHU)N}?|HP^hp4dQFRa$>#kie%- zUgbbyZhVM%v-Iim#WW2PCs#`MxmYPNCLVgKXdY445v<-#}6}LxzG5YEA zm-634^n2uYRzS;!hMmLaE6!d$2Sa}6Xqii5HY{*DnJ3m(RH_K^|=g8$A=gfo0Gc~D}#Dv61lkdN}XNc`lr6vz}f|>lDB(214QwCJ0ftOr= zrw~)H`@DA|f62zG^v~JNlRRLGGYUAh+ak>gC{U02u+)b)J!Qr;_zn`Bu#-Y@jaNLxP@u{rIeA`4cbF{`ZDH#pWR=t%KEJ6rj`BarA|`wyHsZI$ z_h)zb^8s)Q0U$4dan?U$yKuu>+xp1Eo(o^A60*Uwib|f3gvLGK*3vrxaZb6~`=A`Vu-b zJeCxxcwe_nGMp(dRpGB4UH2X-e$$baX`sa~HaS-Q%y4F=zgH+mzsvzgqn)gI5^o9B zhW7CdTHy=F?8~iq>rQ=goXp1sY!4n3{{HJIIo#y_ zQ~~eisZ9ER+gtzd_Tx;rd8Th4B|bf(eaT<=C)X`Osek&?sK40~cKvfqh}$3#7a zjmP1CCcoM7f(hgz5{mzTHc3L3+8x;|0Lj5(1`k<^V4C zyO46R-R>-FLno)Il)wlCY9Y&_BF%A9mOM1o>vSD5Tn9 zg#le%WI92RTSm#@A@0vT%3O9&e75fU%zmuCWZgCFTW_p;y!GW;dPLvkZPp~m)ICuShesbw;k8CJ{<*(W(V)h7(wE&`u~7#wN+ zf+BsasfY9r=kFPEF{FZdgZ@=Gu(qnC}l^ds-!wwemWdGaJHR3y9+iV-parEhuWQ#SH0muNf zY{Ic*8VsCp=b9^{ZFe^RZ%3;s--TVmw_}KqpoIT8Q>GmxQ|t~S($F$dx3Ddkk|kNgzuvyXVSvh&O)xqAQlP zzf~PNU5($%RMP#TqoiuyAsd6^Yytc#0DjwcQhYfiF9StQ>r3rabCxIu+XLl5JI(l^ zT0vEXQp;yqnfag16SR`HczUc${ro$ADK)ovc2o$6LT`_#D`#l5Jw6N$yJ1+5*3l67_gekD^%t1IBS>9u(3INZk6!8VPvI~>9PCpO07 z(Rf`Y05;G9w4oPLm2LUW6hX7NT9-BOXa4<|^P|xc5moNbKJ|puOsNoe?C)M2qb%qn$eac^4RZXk>KL(WgQjHQ40;~J0EGDC(5!9Yl$^p8WC0P`n9L()2%5p`=Ki!US%I7w zNO`wK+vHLN7ZcIg4SVzd^wM{`%JJ<4CBC8CX5{@eO`4NjsPsjXtgzOg@FG3GZa}As zSLl{zrN{Y?H4cT&b!9_OSX9C%wW$}Qt88yw^X6n}vq`md$G|2H?-+~7rX7F^FNGtv z{gpnxN!Fa4Qj*=}@>ANIKq zbz;LoKs(1rY(}^6=5%K^MpY8q;C{4bIYgZzxW_@wAXRm4;ICnz_{n!VgTofaEFtL{ zdwCOAx0Pdb|L*6Gotk*c8PnpRlxd_})gESLX-Dw;yAI`+;R>r0+pDBIRMdkoUwDSH zmeF*|(69UVW6}AT;>~(%yhloCYHNA#AKA(Hs-u-}X&}W$Tk{F(WNHlGGWFn>na&2o zdR9CUr+ec(G>~ek#Jc*_-e$JFi5%6-Qz1l0d5e`RJdwStFZnCkZjpbyt5;6?Mp~hz zgGE}mFLnWSWPQMp!+Xx_L8>^^uV+dfdo^#)58pRV80^jEB-#0_#kDLqh6M#4q;X%) z#@^~i&9O5-;2&QsXBF$J_W*ut*Pm8M@i;S=SSMmJ<-YI-$6lI~$mx=_nv%Pm@n+vnJKNcRdf<8F;48}B8nP~J|f z#Fv9nrLPuDkq6t6envUC)|BaqvYl>E3dm4t;)FV zXYHVLLq@1;^tb5lpYmlU;HfWoMN+}f2e-$VUZ@-~4{4^k$@Rs~eLZ6`*D+#ZBY1Fg zvrCwgG9nMO@wCja7snQ^i%S*fd;XWT#PGEQQsG8@Q$wJ+5aLvFBATv=}G_JJQD zNk2`Bq8lw@u*g+NZGPRat<1}mYq#H7(cF-0-+Q|eURiErx`f{7Kz| z$p;8{SGQ&wKGdH?3oTdtqzHn3MSoVQu6r<3b2U+hn|mc**wu%~c zxIi0hDCJ2#2nVJf-I+2Wp@d)&>18 z$clp?XC(z;KCJNOASZr8-$P0ad9C8V7U*}Jt+fnt+VxDkDT2RTi4Zh=Z$d7eUdOAX z=p|LBZcHR|8JHH?UV5jYj&PlT54kIBdU|xTbu9SVaDONG)a%oZSjxrrz&f*H!tTgD zf4g&n3-LPNP;HPfi5cfgb=w=St?32Li~^erwVAPICvaoSeNM^1N0PwmsH${>eEa*9 z8kjugWGg#erRvsPvy*xIky_^^Ztnr#F=rD|goBWDt+lR`Gcb*^ZF?q%Zz11T^2Hl+ z^s@p&;warz;IHXCPrO6N)9)ObKB}vS)43l6oxe^E`&mr#?hGWWIOwGzU(K_l!Q zRvWGiMvLRKI0wvABD@-FdXc&7R&$M9gnKhg)PcH-YZlX80?GhtoHxz%)b!sPJ;=WP zs(=jWCFy7x!M;K=w(O?cdkf6mL%fHJRMdt8iu%J3C6HqTHXL}mBGs752?jY(5qE7dqV zf(*qbfBI$u&P3?1N}j%!iP%Y1?a==xz$f>c%8Bgm6)uHbyYWJ5i+`Sn=H)PVbB7(S zb9+&|$-?pg6}Y2WaiCAsHvx9v>B=W&+G74a@@c}VRWOga-DtZm3c<-?-1hr5;r>4+ z8l+1%dO~~AFs=#`PBYK3x7hkGh@rfoUt~Li=2Gz6`_1Cp#TEy{#41~>%gdy@N>jaR zX{W3yH>=}g%K$qrgg{ttFZ`W);6>y?I$vO`!_@EcYDeAr`)m)h^u`S>0?KV2Iz3Md zp@l+y!K1d)W%lG`3zid&>wu`1N-VF-Y1Y4@reUvdim5|q;slVrsFybZNp<(23+p`E$u}q7Dx1a2wAywW<>MFD zpHcY9It|jM;v3U|#?fNdo@C>eca%}g6KynxL()6trz%9-csE~fQdx7Q8u*7rxV1XJ6an!h3c76iKx6sc7h1V0^V@;;4&X890rQlpb$iwetm z7YaG8YDv1?N56f9rzZTO|Go)wdHi399wJvgOs)oW8=1mEqa)Y@$+V~t#-^5tosw_K zETr(>VhvT1zkW3}zi0Sj{Q?>YC2vaqzFL2NO=Lwy3pUxhDpMQ;4vmgixCxPRF8yec zIymRnYx|fxsN0FY*eR}RwP~$25XTXNWO}5hXQ+<$IH>c~e`#;3eJ*|VsWR`H+O-k8 zZ<3o;W0lcsZ^r*L)|35+wS*FW`k^Xle`^_}2ccMboo2@!o4frOFLB=|EOUznzKO6I z(1_;)Y{r(oiAQ+Uyt+FsHoS>(_vu|jtdm@8pR+SBd+~h|8$@uj!``-7O)fUxR0u6= zmdYBXbvRwk&qR@+luyrIaL$|A-Hp|&^?9@npBrSf?4#GfD1Ew&6fri-``IDdb}9H% z6OB$-O!}n#W^2sL5M6~zVl#Dh_CnUu>xt8D8;g&-3Vg)N=Q-Mzyvq$1{xq$rHOd#VYs6DBFvo>A~I~P8pS0atxoFQY-U6_Z5Di zENe(wdtZgOZPyd}$0k+5Byq~Tp4sg{_+qArrA&JxYW<~VYdL>(VtO|rPfPcr&`(l9 zSn@7Qw!Ep6vO@{qScBnxZW`^XmKWFGgO}JX&4pSI(jS^azeFwC>PQ8d=oB9F>*VDu zZ}I}>#7fgUtiv?mhB9ITv?<@e=rF3!%WdjK8N(?b(j55zFys%*w8Us<9S`-0sgX8q z_^e#fg~mEaE!NORtm@)E&EEwrdJSxRc|)Wz{#V4H26G0r`fqcRhF$r~(PJv$rhYHf zZP+M(J0*T^`+_ZO$F0sZr5i}=>$>_YHRux8Vu(YOU&%UuX zuFlT$D3k78;lz&GIvgbUyfX4T$!tpK&3Qq1Z2H%2H9AsYZ3Y&QZ zz;!l{toxF-IeAHQRKs2~jhDLpdZFp3OPUfYyEE)- zQ9pYG-P?*ztC}HEc$lI3)xFc-w?>*oyiIuhCK2FzHFNjQf@ylgqk^?-Q2a03^N5j= z5o3Lk+eo;gxbGWCsU4aN-3yUz!8rvfs2=N%&&ioOUB{^8C4T!kP*_eyzRBfYdKfr= zQI{)pG57`CxbR1R$i;d^M8?9tbXPPgang^C)-G~QTdslU=bkPZT-BD%C)$|*u1}p< zQ!lRD#n#PmMqWVMS96NHPe#arF+IDTxj-E-I}tLh{n_KWCwD^Lr*|>NCohOiovQly zQ^^cW#=(LvWnv4c87hLL(613Dm-j_dO}{KzRYVfLJHmR$V@5-ZWs^!UjhGrajZ1!u z36q(Hy`bRX75CsC=z9}UB$G$(S(%9)#yu6d&h|kwif`6=3Z}EL!iG1M1OE+*Kg_`8 z&MEY#Se<$Ich)_FihE%)=dG7Um|B#cE~$Tr{ewzFI7O zE{UB#r0VvvjF~`G%nVb0$V8{fWiZ%RbayY(E|&44*=`oyFG zAxDM9A5g}uH$kS1`)xcwWb`#(-1p9MFdwxX<~Mgl%k`f%?|sXq_;)_s&lMj(L zR(D|eE3S#$E+&k~)Rp~m*&mE^oEE)eIn%$WNp~{UZ5E8L_F5>ukLvQlp?%mAE>At# zb{AhD#{nMEfDR^?+~GR^U1><&c(oyPpX8vMw1@lyaLwO3+dI5Vt+7%#y+en1X{3*` zq-agl{^q*f9%(q0r8pXvl}<2&EYO+RvrQqkn^V`57!(2S-|vzXH!CBy`Q|RF<&1SH21QEM1&uQz4QOJ&dHvc=O1(li;hGxLPc+;XV=f!xd}OMcuEb8a z!E(61zoNmB4Xk~h;bUmS)RDlwX~S2wAYwv5mf>k)XC8JGr~cJ6jL#}E!qooA5EL;*aM`dmVUG+_lmhBD}u0xkeO8XkVDA3(wf|I)y*-h;2?f5k>KO*x!yL;PSFeK-L zQ3csefy6%+88#eu{>57U#-xo$(5%-|LMkEeqn`6WXEk?_2I~5kQB?GpdgNQ0Fn<2> zsj4C5rW~@Ebk)h0<079Z&$ziSWH)JJJbmkc<2I)iWg0HA-74t zta7^{j%m3fyD=bIE|cBJaG#awdKTBd;NCpkl}-Pr{YtT+)|jT_RoNzmvAnVj8JSq& zdbQ7n8CH5bhWeNdL{Th5Z`%L;Xff2`9Lx?2^w7cI|JtGQ>q z1PF2@EJ`|C%%D1E<{BPN*sxP?8D7WruXO$u!WaTt3j1_;8`vi^Z71yoTKm59g-W#N zrXsUKgsz8?5nP}X3jXo1Tkrj~q%kh5F!0gJIuQYQPqjUC3q z8pi*4Yw*7qvf-ziL42I7dy{SY!i)AwFiR8VShyww-}qMDx2=Em!-AQuu;sL^# z>Y+1)7N;OGaXLA+?YrZBhv(IB9@kF1YJ3oVN{<@5PK)SQ?h)z10p>@%8kRidBQcc( ztx4d0+4G^5Fk#JpGH43_o*rTsz!g)Y$$`2Qlb2{7<33Hgtq))53$7=dzmJqQjjfdF zPDt8BYg0~3omWts#qY{D*0Pz(%AFJ|d-Uf{4rvCTEhWa-?q5k&-QJaUFGB`&gv2t) zIyWj94e$4<%#Rdy{e){-y2i+j>8ixTq;xsY0Z_dY8u?zyVwR?8vYt>Jk$9$!I_r!- z#|gE|7W8uh^$Av;poG8Wshhdt@?BNJ_3fF2M|XsJ`@e7ZZci6+fP%V(5&(xEMp&u1 z@E-t1jWDlP$P=SvK?~J$0}4b*u{&mK^YYJBoeq2Q4CSf}WrymGhl{Z(JF%ILMzW;8|Moa$^SVIbMkFdqyZ_gK!b~OcRIZScohW_T$d6msa5^|6ndXq)($e{|9|Z-v zq1FXN1$uYzoE$iBYLNPOsLfN*_9MkY{2B33MdZGm=&G_bWFJ+N=VNv&sUwBmwHd4+ zQU~6^R2p!K54h79I^XCggFJ**NrI-F#UksjJA#7)!W0fr_!}mkzIj~KvR+!NMgcrM z(jfHA$;?XJWLTQ(4VOF9RA@7Q4x8wE`o5^8M<6WJ#Ry#4 z8KuSpR8qX4Nw<0%#!YPY$BKr2u;FmVExb$BG&>wr>{WC@JHhLb=e*lW#j-{n=zZQ# zoSNQN>2c!jkH0VR$Cu8E2jPt%Yj+&uL6x)A`NpG2w>GbYgw!K0;l4+417F0O!-CxB z5LH_aHO4PLkmM64AlO>}9r zxtV~)*1Pbshn*{;g*i?$Y#h1XO4FTmJ^h_}?+iix0A--uBQ#qu;Y)VQCB0A1jj>#J zw+A*+$u55hJ{9v+U(FaA9AZUQF%=J|cG$X}SbkZ4ulMn$zzQsOo^Tc|H`_AGiX2XR zNJ{fc>K#JO#5rV{vLNVMbBQ-(WUbOo!+3=@%e_R5>0>lI%$kk#yS?KERK%HLo0XVO zOgx2lL|o+iMf1Av&V!@A46&Gg9pn76tuMB*8;E=KJt|x`ZvF6W!PFA>jnkqT)(AAI zN==ZEZLDfu#IY{%&^r!sCS^ z_mj<=f`}br5w-U~R45q*w2O*(WX$cy+oX247BYdNKm7Z}e;FPX6;iXVuRP?=Pi`ZB z4Ya=QT~>aeV2%r_acAvQ2ahbRwsb@M-_9(_>_^fGCGgU@EFQZ7y@B9QqK9c@3x_me z@IOWko1G_PHo2HK?_Dx06zhJGpQT|<2wrdebSL2Qe}W8?KH+%vR$%pXdeo_}|xB4U(@i(pf{d7ZXq{cnrA>Lwm_n(n58Zl}OXZcNc( z>K;-)O~VZD{I7&|U?3F>{N`g{_AYj&5nQytK66nA(WCHY{L&8iX+5IY_FqY<)6Ti6_DR#n`ETO+iMx zGWGOIqSyZ>TTskPmSTye{U3hzgk=Rh;_VT?F|GR2CATo|=KnWHZ$@w2{N$l=o%tU; z)lr327T|%zu>>wWH9Cb6jh&G5{x`0D18-B;^Jaln;B0&kX82zmgJ3Lb=vBI<%$_Nl z^Ve?ABggQ_GDPB>8cw*EkJ#eZr~MBlGsZfKpvBK#;-B8heeKiuJ)UYH6hHBW@;-gW zERLNwz0X7koaojea+IGTK|vm5o{58>%tx%cJ1K%ctyID1%8ktR6WVUf8bIy%@AR@v z#A9Cs7uL{z+X#x)LNMH@e5d_bt|+NmWxw%lsH1IiT+Yi%$$$IzZ@(aTX62~NO?asd z+i~=w(nxLP_Ef3%3q!#LQH5n#iVw~P>`aZbi!|M z;IyTZ%ph-A{bHh6XQGUXcYS`9n5i;CV6-evFlIW=hV^INz^LDg)K_;rNKBg!3$%+M z=_idgu zC1MjJ=wDmnD)5~QiF@JD1z2gSnB`CtrezgC(Kz;2pH!NmWdjL+4Sw8Ry@qarOgn1B z6K&mNDbC(-w$qkl#I-&uv}|2T4C}!!Qa@Tb*we8E@9|RIvIbrFZDii7Beq4N*ahh5 zU-!2l_q=%ISBvxS)|nms?a{ZBO9M_G68qPMv$LVb_>q?BTx(|76n@d$VmFRbNykLVa?e-c2VqB%o9-@NTna_o*&+dF(!eyw*D5YWN}urG`K$HYvHBZOp`@)H2}oc4*W;yOvZhABAJ>ES%<|HVwGmF3>PzX!!B{Xp%f!YDkkr zDamD1n+pbQ$17krr`^?$wTrf5bv0lR1)MBDE58kb({Ri4^TN9yK&O_j^fXFM58U1t zhm8Et^IL6iDyixAfGNJjmbOa%Y9l)JHdmf_g#2MlD9(qAhnE<1MC` z_vbBOQCldqCP5J`LPB`Ka!;O0=#Bv#484+~1mrr1xe|Nsco;yu{aQFQH zA9UkeT4kh;MMb>#N#LS!G%7R6Bqx7-vLQf{}$yju-Xn0_H%>3SM(+J z{a}l$gT0e4l!{x|7F$kk{RBAZkW#r-5*1*=`cbw^H9L6V&L*eg`5o2J_hxB+Fc0o12Hm^9cA{L}X}dYOn3jS)QiiYY=5%HLvnzlHAb^K8fH` zzpgwH5PWgq7lo2(!~IbDezgZ!B??OJ-ut`8s<3~k5t$JunVLBxAgnk)4tp>YK*s|IsM&i$P&LEwUa=DwTCBP~pC85=; zDcH2vPmQwi&JFB z2pk@=@I@LXUpNiXz|v~wkrqGrhP57J5Yb?ZCyJMz0WPatZJ=v?3h}Os!j4cKL0dW} z*BZ{;KjlPp-9t1ciP1xp=%r zUoOd2o;?k#{^{a1#9EC1c_B){Kw(wGU`=$oD<<_Juieb4Z~ zFqeyYBN}78`K_(x^t}dM%8QDz7KgBSdqt2r2sHd94QIWUjE_ge>oEU z1$9^5v1kkzba91{K~)Cy^~7DM^p}8!kHPb_nX@_%Gtl#hCEv+K1K?V(*|ZSd>^P2a zzbKH#-iHX3y=zOpol25)gRjPyTpF4{2V$}E(cX_-#g1a zc1?GwiHn{qg;XD(o&c3uI@xOwa7cVkt%teTO8e9YkRlA)stqMOU@#A_uB~AcS@Cqz zTN=1x2*+4VgLK7)8avl-d9gPU<7Lo+(&p516O{A_#HnSq&%?`@ztib-x|X&!x(`Kk zkE>gtJHM5dt_We7@zNT4J3B|%2zh?%cATDn=?*;MmG_BHOy86#Go<=ghuqJk)}UM6 z2t8u2yIM~eu{sfmX#7LbH-(`4!h)G1G#QrHZV{@iPuI}hY^wjR`_{9`0!f6M5$Vr?wz z{q$tepR?^4JryI93JE)v3h*n42?jyF15DXyD1{g#Pk1cm<5Nf zA2ojR8nrOi@>YqS)cHHIG~hv2{D)U!m1~~)oT*NFu!qH&s;bAN&#`(IJaQ3~x-aHc`(^N34_bmOr8k zup0|llGjHaVhV)8VQ`KuuB(_TOsLos^waJEgH*!kRB{YkPUcHl@YFqs|4GZ^pc-t6 z^R?Su~sLAny|J0tr#1umo75 zL1&%oQWAv5Z{bir?KMPSjxF(}J-I=+!9^1$7H5X(yzw8fz%};NG`PBLg1KKZ@)22R zvhw1AEL_(+i*^TstWwcPMkMJeRzz>F7iZ$ege{(Hi@7mqGahZH^Z-ij4!2qOLU>}Z zQnunm{a{uOx~G6-o-7lYKW^Nn9a!jBc^vj*#xB9%FGfB76@Gs}md_IyxYUtz4))eP zMUsTZ|E3Y01R_Zs+AY)}bVv09_BekWN-}i~AmLcxfKdXuDuBk7{SdbTiz{N_&CZPL z#t8LwM_F4X)rN}B>LDg*(NhDj!oS3Du99l%)ZT}$>t+cB6TkF(1bA7Io zxwcAg-1I>0He8c&`V^6`;~%mFc0?&LS-(il!8_x<&8PeN<}aQdc(pJ9?1vvpO2q+z0a29b{+ zHq5inWL9mM7;%+i8aIF3Rem}5Gh!|aC`#0daZ(O%U^JXv@Sw;4zRFB5b^aOGZ9r)b(atNomrKIPG6k9T#%O0jzQ;b z5hn9mGha=r-_->-OJpk}S@?O0>~VX=bHXwSG2iVLI{-!%6-*!@mKZ!`VewM$!rQVg=xq@FlZ~894=yB@`@T&( zHCkQ}bdPh0aE+RoH&{8lCDg0?vEZBkjNb+6$f7w_TUmTPmX&!OTG$&JIf0G+1|MIG zkZ8BpM%Y|)Xs+A3<%za8*>;WV8zlTlx;!b6+7ubWMThK3Gl1`(&)sTa{!&kR)iAXBm@u08QymtA;*@fZNbGDeMXEA6p86a^30*z)TQsE z&|hjx#-WhnEi;r)8+lL7Fv+Rl2HZ_lJRy1jXqZTL4bs3qAP%Zp;z-2SbMSJzRMH@- zL#bv`q!Q`HE|~DpV0JD!i4nNZlp%1Ag)e=tp5)sA_aB{ChBooBSYsh0-~`5b32?Pi zm%zKF+9Qn&@-Q$HB)>6IKW3>yne?8tYV$4)hl%imzu7(Wu_&hKeEUsJ zx@Tu|}8y zjm8mKil16c_7Z{%|G-zj?K$O>gcDT_-~y}<20dxx0dp5OW+#X^23T8HY70tF?`hEW4#MZZ1D zKu8X^j9r`;gLKN@ZOg&QVG09{7q`&O!st_3M9T&9v#Uo; zC17J9r7SO!tXRL*yQ!`kpMZeiuf_eJ5Kp42DN2< z`0zp9h&t`<_+FbdET$rkKZEM>pr#%793gPn)RaUC$<4Kq_WL~5wU@!! z-_1JCeIP&nBkvWH0Mz1e26aQYbYW7-!KwehaF-6aRi1dJe)7%^yLE2CuaG(KMJl8` z7~X~%u-L&(WIr9v`_hg3zdGG$2#mf|nb?_M2M?tEUkFS3vDjz*wkkTa;=iiaY3Ki_ zS|hg|f&DG&Qr*`=eE)@}(s3#;Pyr@BEq|$It6l=_*WfJsOhAYdK)7_*AdtER=KRCR zC2{ow_oLo;j~rzC!>IYLl%W0U`~P~&qbj-TS7m{6^9&^++|X+EE&roZb?Nuj8v{)m zA-S%2)(@i+tME2WnhZoVKSAC^Zi@7C3$keM7@XQK08nCItKL}KJ(hbVT~(z31ExtNQAs&JUPFa7%l zno&oVIHe4YBduCXDpoL9>O<$bN<7&dBdNz>vx`Q}H`PQwi%IFve#Lv8ot>Ig8Z2>S zQx0q?%6fX~RI){CC6$hZybH;E7L|^_DqO`H?B)gNEV3#sEJuEQFrTZnavtoB2Up$x zI!ZE)apzl#wWc#ZY@e+hA#v_EN(y(Dsd%DYAQMGLYx-CusOk=0JukLR6LBFLq5L(d z!>6aL{5(>4Cm-DlE(^dWAU6&sT=aRk<29lbu+}lPJailK@Rxps6$&aM>BXUJdPqS6 zi?Lk!?{-M&YJY~fIP&(muN^590g6wvp@NiC=zH2BqPvcNFADQ5jZKw(i{EZPy}M_c z3CAh+Wz+Rh&$)nr^WWReg;D{B?=(h?XribSIFD$e)^n*=C)A1563o7c9@LslOp|ly zBvQ?n8=!blo*PMKJSa${xl$Op(4y~5fvedGE!5Lhqt;T>_3mt{ipOzhoJAzz!>0!; z1;E1hlqGV3TfuI_>79xG)T(Y-U9ttP6i2V?Q#_CT8g)|*wdw7#ykMu@NfWa1-04h7 z1kX^p``HecNpD=a=hg9Wv&Y4rXo?~Ka;`!eD1l8=InQ97RRm-SQSxB~<=I6TsZ26u z!&}iE1b!xH%`u1V(c~d^P&dykTllY$nfY-pG4<>eUbDfV4b$+S-n$dN3cC!h8UT;y zw;#Kk=yR}`W{^0=<2U^n54i_|e70XoA)M{nl@D4@NikQYPT19+*GsySi^8xn}Caj@i9LS)Z;|JtS=3K45TT-6hDf?ulnjp_rNv4e09?6z*eH;fWoQk(H zv{;>z$M|D;hEZqTyVmQSqNGO*Y=;pNI;H2q&ilMMz>DEVtZU%VW4&-tRLnY7F+0Q% zdQL2+_cO5!XRCpqen1~3e7S@WfIQ^Nr;szrCH@ejJoVn-wq0r~F36WTPi07%@D+f) zek6^r-ktzxWkH#k;3@=>nkbOLoj~lzoSJcj9D)>Fx7-!gB?N2p9WK>vD%Q-6w(PBE z`?YED!>BtFc(O6j_sPmVn9kiiA-8*%G(tVQnr-B-C)c!bwJ1N{9iLcp88KnRUK@b8 zKXM>VL_kDO|C&W~%r|le@-_-Uqi`-?Nd_u^9=cK%FI<;ICiayE@|O>^N?mF&>v)Q+ zTW|*(StJ{RB>$Wmx*c<((5`ZdO@^JJz%nYrI!<%aiIER71VCQhzC9_aA#E)!P^04! z!LDqr1i-sv-Ugjf>L>QZ<=uanMeWy*hJAz(fs02UKdz<5oGq^CDGn%{Ok-$mxUL(* zR(GaFl)Xzvqx2naElX7kQD7@VkiU@*|J%DI5lY7vYWF&5#-&<3zwYGP@<|hI8E^W3 z7nA(=tl@U1mANJN^6`3Uyp)(NY|lmGDp|WfL(IrKa$I4W4_{9)i0njsv!aFS%MV*_ zg0iH7O@oXXU0zBK!CL!_LdnE7VzD7;@sp{domc-RmDs`#D?O)bSSkn;WU|Mf@d>8< z(vvcZvr=&r$TErqnM9mX&ro7+wu?mZIoC?y-(Pl%>jjRsKKYm2cwADhRx9{)`D`}h zsK=UAo^yRnZ^U9fa?7o>doK!Z3rg7BY8Q6g`azXo;La_U0U+$+iwueegM3lf{ELQT z!%&iRo@B=|^{Go}K#5`2!t^k;}Oz5B)Cs2}38ybo1&ic<&18yM7cb!tM;fJvqGupDWyi7t&{{s zj7T)G8fvvdMeAzM*eg}vxV!!ZpU>;_!}ENe=l#5&RYI`QpAFv&SNtcW!$3Lc*P)5* z@O&4lfQ&@+Bt5z~a_-GDZ-pA^uE45dUbzq`wkRR7PW72k|293MS6xqBp;;v+fP##O zLIP(q6*~K2U4e>lZ!5((ba|P2)UdjliI!I1TiY^D@Ozc3uQ}y{-luuSaQZ9wMh2$t zZ&kt zwpMkOO6@z;*C_mlvW%Vqj9yXvNETz@iRD_ih7y7JBYt=4Lxg2Z+cBj%a~^yp7^r6X z&i&@%a5Q@6G}dqeBd6mwFnS0y{xLLZhzfQnLwg|f;!DCnuSu>S+GQW$P8!cgj6waU zyFD$d&%26aa6SW09s$0>6T5!Ab(h9wzz&;h+>i%+L89qbH=N`PYOZnk-cMsTY`GjB zY}3%|c}0XOeqs^LUM#6U$_rn$P_`+0{aF8)OKz?i?9Vg0sd+I?Bput4In?I5VVfzT z9<;C6Bw-XKk$kR4+1znEsQ+#~w zy1}V=vVh^}j5|!{uebPI^m;%nq^5udbWY+RcI?%KHeK~L4+HM6N`2I384)%68xkIe~_nf_VNtG@aM#vUBMWMrYzwpyfmIc2(ZFq5d6kUg5_aWqIPIU3_CTY z8$2`x%>N3kVcY~>=g>6md?yXdvQr3EP9G3{1kjh?OMytvcG+yX49OMxxcVfAH9kkdHkYZxZwQpL)B+nMpm`0% zP+NmzEM z*vT`1A>`c<76R)^x`itloS)gUD_xl{c4s-;rx>WBQpPuz5&M1?uuSwyvoZid*@a5} zf;M$^vx}p9H2W}QQwmS&c9Y^g!1(+)(lHlCCkZ@|8Ia##nFDiV;*&BMgqlKDrJL{F z4T4YBc%jgSVTq~94dp&Iq7ds`jE_WuaXlVlnFX{#~myqfgQH{#76G z%Te12OoMFYcz&StT}#q?*)XZyB0^q^BqboKXmA?c;niP7JK;~+n3^uj6up*t)>=I^i3ZKkYUqv_hle3rIMq&rbUkTZ_ZzxRd$z zB)Dl)-HvM26U)4%-&Ti!YbD=4feiRG0bo2hE%86a4vbGFhX9u0e_gs2C-(D*V8=Va z5rH*ixwQlJUpdliC>qy#j6U8diHb{YQ_TPI>IT@_W1xK9({iEB#sbUkhz4nUyUwS@ z08{zAOfX)-jlA&iO~mL+zH~RZRy%V-m@BzJ%0mGEdhyb$u+2NO_4RToV9{SL%8n5= z$ovH^AD80c(DiGETt5QuUuvg3DpcQd6+xW0@M5PHb}==7xEzPNeXE7XrXr2~NdMhH zsp%=6Ay+su%cb)Z=;!wM0n@uqVy1~e(>?7;X3Q9G75%q9Z>$*d$59&q{M^7K5irHZOF{$pFy-EpfJwfYqaWnMO>;rcL+c$ zn(iKoYGMW};S!4ee&~C9G@y;Abf!@%$w6mNr&BO_myBOuJI~(6A*47iv!+7e42uo@ zAz&BlQ2-NWy2!`V^ijgh@yT^~^uj%ICb>56B)Tg02z_hvG^nb{S3E+n5F=N6sIe!w z^}i_?ppimvXZd7>vG^jm?#DWkPA&Bawd1Wc8*xu&9S#pAL8O58+6;=b!x)BRN|BT1 zd|##=zpxg%Cvr`k*u4Y)yUQU@`$V;xh0mgaxk!e?DVj6Njvyio?@5)UJVOE{QiTQk zHB3|II+IzdX5J#Vvlpnf)1?BmbChR@)2hS}tUDE=$``9iRga9c2B;IQz7Qm*vdtKn z8DBkg$EH(cjpzPcljPGmRwY$!zmgjAqceOqs8_vHOVK-grD?&MSHad~*nYV7n}`b5 zBkQrO9%@&nbY{O)h+-mh^ciW@tG?P5-f`Jl$HG_@OY+lg^z2M_-ej